Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Improve][Transform] Optimize transform structure #8071

Open
wants to merge 1 commit into
base: dev
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

import java.util.List;

public interface SeaTunnelMultiRowTransform<T> extends SeaTunnelTransform<T> {
public interface SeaTunnelFlatMapTransform<T> extends SeaTunnelTransform<T> {

/**
* Transform input data to {@link this#getProducedCatalogTable().getSeaTunnelRowType()} types
Expand All @@ -28,8 +28,4 @@ public interface SeaTunnelMultiRowTransform<T> extends SeaTunnelTransform<T> {
* @return transformed data.
*/
List<T> flatMap(T row);

default T map(T row) {
throw new UnsupportedOperationException("Heads-up conversion is not supported");
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.seatunnel.api.transform;

public interface SeaTunnelMapTransform<T> extends SeaTunnelTransform<T> {

/**
* Transform input data to {@link this#getProducedCatalogTable().getSeaTunnelRowType()} types
* data.
*
* @param row the data need be transformed.
* @return transformed data.
*/
T map(T row);
}
Original file line number Diff line number Diff line change
Expand Up @@ -44,15 +44,6 @@ default void setTypeInfo(SeaTunnelDataType<T> inputDataType) {
/** Get the catalog table output by this transform */
CatalogTable getProducedCatalogTable();

/**
* Transform input data to {@link this#getProducedCatalogTable().getSeaTunnelRowType()} types
* data.
*
* @param row the data need be transformed.
* @return transformed data.
*/
T map(T row);

/** call it when Transformer completed */
default void close() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,8 @@
import org.apache.seatunnel.api.table.factory.TableTransformFactory;
import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform;
import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;
import org.apache.seatunnel.core.starter.exception.TaskExecuteException;
import org.apache.seatunnel.core.starter.execution.PluginUtil;
Expand Down Expand Up @@ -144,7 +145,7 @@ public List<DataStreamTableInfo> execute(List<DataStreamTableInfo> upstreamDataS

protected DataStream<SeaTunnelRow> flinkTransform(
SeaTunnelTransform transform, DataStream<SeaTunnelRow> stream) {
if (transform instanceof SeaTunnelMultiRowTransform) {
if (transform instanceof SeaTunnelFlatMapTransform) {
return stream.flatMap(
new ArrayFlatMap(transform), TypeInformation.of(SeaTunnelRow.class));
}
Expand All @@ -157,7 +158,7 @@ protected DataStream<SeaTunnelRow> flinkTransform(
.getStreamExecutionEnvironment()
.clean(
row ->
((SeaTunnelTransform<SeaTunnelRow>) transform)
((SeaTunnelMapTransform<SeaTunnelRow>) transform)
.map(row))));
}

Expand All @@ -172,7 +173,7 @@ public ArrayFlatMap(SeaTunnelTransform transform) {
@Override
public void flatMap(SeaTunnelRow row, Collector<SeaTunnelRow> collector) {
List<SeaTunnelRow> rows =
((SeaTunnelMultiRowTransform<SeaTunnelRow>) transform).flatMap(row);
((SeaTunnelFlatMapTransform<SeaTunnelRow>) transform).flatMap(row);
if (CollectionUtils.isNotEmpty(rows)) {
for (SeaTunnelRow rowResult : rows) {
collector.collect(rowResult);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,8 @@
import org.apache.seatunnel.api.table.factory.TableTransformFactoryContext;
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform;
import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;
import org.apache.seatunnel.core.starter.exception.TaskExecuteException;
import org.apache.seatunnel.core.starter.execution.PluginUtil;
Expand Down Expand Up @@ -192,17 +193,17 @@ public Iterator<Row> call(Row row) throws Exception {
List<Row> rows = new ArrayList<>();

SeaTunnelRow seaTunnelRow = inputRowConverter.unpack((GenericRowWithSchema) row);
if (transform instanceof SeaTunnelMultiRowTransform) {
if (transform instanceof SeaTunnelFlatMapTransform) {
List<SeaTunnelRow> seaTunnelRows =
((SeaTunnelMultiRowTransform<SeaTunnelRow>) transform)
.flatMap(seaTunnelRow);
((SeaTunnelFlatMapTransform<SeaTunnelRow>) transform).flatMap(seaTunnelRow);
if (CollectionUtils.isNotEmpty(seaTunnelRows)) {
for (SeaTunnelRow seaTunnelRowTransform : seaTunnelRows) {
rows.add(outputRowConverter.parcel(seaTunnelRowTransform));
}
}
} else {
SeaTunnelRow seaTunnelRowTransform = transform.map(seaTunnelRow);
} else if (transform instanceof SeaTunnelMapTransform) {
SeaTunnelRow seaTunnelRowTransform =
((SeaTunnelMapTransform<SeaTunnelRow>) transform).map(seaTunnelRow);
if (seaTunnelRowTransform != null) {
rows.add(outputRowConverter.parcel(seaTunnelRowTransform));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,8 @@

import org.apache.seatunnel.api.table.type.Record;
import org.apache.seatunnel.api.transform.Collector;
import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform;
import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelMapTransform;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;
import org.apache.seatunnel.engine.core.dag.actions.TransformChainAction;
import org.apache.seatunnel.engine.server.checkpoint.ActionStateKey;
Expand Down Expand Up @@ -113,9 +114,9 @@ public List<T> transform(T inputData) {

for (SeaTunnelTransform<T> transformer : transform) {
List<T> nextInputDataList = new ArrayList<>();
if (transformer instanceof SeaTunnelMultiRowTransform) {
SeaTunnelMultiRowTransform<T> transformDecorator =
(SeaTunnelMultiRowTransform<T>) transformer;
if (transformer instanceof SeaTunnelFlatMapTransform) {
SeaTunnelFlatMapTransform<T> transformDecorator =
(SeaTunnelFlatMapTransform<T>) transformer;
for (T data : dataList) {
List<T> outputDataArray = transformDecorator.flatMap(data);
log.debug(
Expand All @@ -127,9 +128,11 @@ public List<T> transform(T inputData) {
nextInputDataList.addAll(outputDataArray);
}
}
} else {
} else if (transformer instanceof SeaTunnelMapTransform) {
for (T data : dataList) {
T outputData = transformer.map(data);
SeaTunnelMapTransform<T> transformDecorator =
(SeaTunnelMapTransform<T>) transformer;
T outputData = transformDecorator.map(data);
log.debug(
"Transform[{}] input row {} and output row {}",
transformer,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,23 +18,23 @@

import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.transform.SeaTunnelMultiRowTransform;
import org.apache.seatunnel.api.transform.SeaTunnelFlatMapTransform;

import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;

import java.util.List;

@Slf4j
public abstract class AbstractCatalogMultiRowTransform
public abstract class AbstractCatalogSupportFlatMapTransform
extends AbstractSeaTunnelTransform<SeaTunnelRow, List<SeaTunnelRow>>
implements SeaTunnelMultiRowTransform<SeaTunnelRow> {
implements SeaTunnelFlatMapTransform<SeaTunnelRow> {

public AbstractCatalogMultiRowTransform(@NonNull CatalogTable inputCatalogTable) {
public AbstractCatalogSupportFlatMapTransform(@NonNull CatalogTable inputCatalogTable) {
super(inputCatalogTable);
}

public AbstractCatalogMultiRowTransform(
public AbstractCatalogSupportFlatMapTransform(
@NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) {
super(inputCatalogTable, rowErrorHandleWay);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,20 +19,20 @@

import org.apache.seatunnel.api.table.catalog.CatalogTable;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.transform.SeaTunnelTransform;
import org.apache.seatunnel.api.transform.SeaTunnelMapTransform;

import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;

@Slf4j
public abstract class AbstractCatalogSupportTransform
public abstract class AbstractCatalogSupportMapTransform
extends AbstractSeaTunnelTransform<SeaTunnelRow, SeaTunnelRow>
implements SeaTunnelTransform<SeaTunnelRow> {
public AbstractCatalogSupportTransform(@NonNull CatalogTable inputCatalogTable) {
implements SeaTunnelMapTransform<SeaTunnelRow> {
public AbstractCatalogSupportMapTransform(@NonNull CatalogTable inputCatalogTable) {
super(inputCatalogTable);
}

public AbstractCatalogSupportTransform(
public AbstractCatalogSupportMapTransform(
@NonNull CatalogTable inputCatalogTable, ErrorHandleWay rowErrorHandleWay) {
super(inputCatalogTable, rowErrorHandleWay);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@

import lombok.NonNull;

public abstract class FilterRowTransform extends AbstractCatalogSupportTransform {
public abstract class FilterRowTransform extends AbstractCatalogSupportMapTransform {

public FilterRowTransform(@NonNull CatalogTable inputCatalogTable) {
super(inputCatalogTable);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
import java.util.stream.Collectors;

@Slf4j
public abstract class MultipleFieldOutputTransform extends AbstractCatalogSupportTransform {
public abstract class MultipleFieldOutputTransform extends AbstractCatalogSupportMapTransform {

private static final String[] TYPE_ARRAY_STRING = new String[0];

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
import java.util.stream.Collectors;

@Slf4j
public abstract class SingleFieldOutputTransform extends AbstractCatalogSupportTransform {
public abstract class SingleFieldOutputTransform extends AbstractCatalogSupportMapTransform {

private int fieldIndex;
private SeaTunnelRowContainerGenerator rowContainerGenerator;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@
import org.apache.seatunnel.api.table.catalog.TableSchema;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.transform.common.AbstractCatalogSupportTransform;
import org.apache.seatunnel.transform.common.AbstractCatalogSupportMapTransform;
import org.apache.seatunnel.transform.exception.TransformCommonError;

import org.apache.commons.collections4.CollectionUtils;
Expand All @@ -41,7 +41,7 @@
import java.util.stream.Collectors;

@Slf4j
public class FieldMapperTransform extends AbstractCatalogSupportTransform {
public class FieldMapperTransform extends AbstractCatalogSupportMapTransform {
public static String PLUGIN_NAME = "FieldMapper";
private final FieldMapperTransformConfig config;
private List<Integer> needReaderColIndex;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.seatunnel.api.table.catalog.TableSchema;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.transform.common.AbstractCatalogSupportTransform;
import org.apache.seatunnel.transform.common.AbstractCatalogSupportMapTransform;
import org.apache.seatunnel.transform.exception.TransformCommonError;

import org.apache.commons.collections4.CollectionUtils;
Expand All @@ -44,7 +44,7 @@
import java.util.stream.Stream;

@Slf4j
public class FilterFieldTransform extends AbstractCatalogSupportTransform {
public class FilterFieldTransform extends AbstractCatalogSupportMapTransform {
public static final String PLUGIN_NAME = "Filter";

private int[] inputValueIndexList;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.seatunnel.api.table.type.SeaTunnelDataType;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.transform.common.AbstractCatalogMultiRowTransform;
import org.apache.seatunnel.transform.common.AbstractCatalogSupportFlatMapTransform;
import org.apache.seatunnel.transform.sql.SQLEngineFactory.EngineType;

import lombok.NonNull;
Expand All @@ -44,7 +44,7 @@
import static org.apache.seatunnel.transform.sql.SQLEngineFactory.EngineType.ZETA;

@Slf4j
public class SQLTransform extends AbstractCatalogMultiRowTransform {
public class SQLTransform extends AbstractCatalogSupportFlatMapTransform {
public static final String PLUGIN_NAME = "Sql";

public static final Option<String> KEY_QUERY =
Expand Down
Loading