shuai-xu commented on code in PR #10362:
URL: 
https://github.com/apache/incubator-gluten/pull/10362#discussion_r2289714778


##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java:
##########
@@ -441,9 +445,11 @@ private Transformation<?> applySinkProvider(
             applyRowtimeTransformation(inputTransform, rowtimeFieldIndex, 
sinkParallelism, config);
         final DataStream<RowData> dataStream = new DataStream<>(env, 
sinkTransformation);
         final DataStreamSinkProvider provider = (DataStreamSinkProvider) 
runtimeProvider;
-        return provider
-            .consumeDataStream(createProviderContext(config), dataStream)
-            .getTransformation();
+        Transformation<?> transformation =

Review Comment:
   Please change it back if not necessary to keep this class changed as less as 
possible.



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/sink/VeloxSinkBuilder.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.common.sink;

Review Comment:
   This class should be put into gluten package



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/source/VeloxSourceBuilder.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.common.source;
+
+import org.apache.gluten.streaming.api.operators.GlutenStreamSource;
+import org.apache.gluten.table.runtime.operators.GlutenValuesSourceFunction;
+import org.apache.gluten.util.LogicalTypeConverter;
+import org.apache.gluten.util.PlanNodeIdGenerator;
+import org.apache.gluten.util.ReflectUtils;
+
+import io.github.zhztheplayer.velox4j.connector.VectorConnectorSplit;
+import io.github.zhztheplayer.velox4j.connector.VectorTableHandle;
+import io.github.zhztheplayer.velox4j.plan.StatefulPlanNode;
+import io.github.zhztheplayer.velox4j.plan.TableScanNode;
+
+import org.apache.flink.api.dag.Transformation;
+import 
org.apache.flink.streaming.api.transformations.LegacySourceTransformation;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+public class VeloxSourceBuilder {
+
+  public static Transformation<RowData> build(
+      Transformation<RowData> transformation, ScanTableSource scanTableSource) 
{
+    if (transformation instanceof LegacySourceTransformation) {
+      if 
(scanTableSource.getClass().getSimpleName().equals("TestValuesScanLookupTableSource"))
 {
+        return buildVectorSource(transformation, scanTableSource);
+      }
+    }
+    return transformation;
+  }
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private static Transformation<RowData> buildVectorSource(
+      Transformation<RowData> transformation, ScanTableSource tableSource) {
+    LegacySourceTransformation<RowData> sourceTransformation =
+        (LegacySourceTransformation<RowData>) transformation;
+    try {
+      Class<?> tableSourceClazz =
+          Class.forName(
+              
"org.apache.flink.table.planner.factories.TestValuesTableFactory$TestValuesScanTableSourceWithoutProjectionPushDown");

Review Comment:
   This name is under test since in test environment it loads this class? Which 
class it use in production environment?



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java:
##########
@@ -466,8 +472,10 @@ public Optional<String> generateUid(String name) {
       } else if (runtimeProvider instanceof SinkFunctionProvider) {
         final SinkFunction<RowData> sinkFunction =
             ((SinkFunctionProvider) runtimeProvider).createSinkFunction();
-        return createSinkFunctionTransformation(
-            sinkFunction, env, inputTransform, rowtimeFieldIndex, sinkMeta, 
sinkParallelism);
+        Transformation sinkTransformation =

Review Comment:
   add `// --- Begin Gluten-specific code changes ---` before and after your 
changes



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/source/VeloxSourceBuilder.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.common.source;

Review Comment:
   ditto



##########
gluten-flink/runtime/src/main/java/org/apache/gluten/util/FlinkRowToRowDataConverter.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.gluten.util;
+
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.conversion.DataStructureConverter;
+import org.apache.flink.table.data.conversion.DataStructureConverters;
+import org.apache.flink.table.data.conversion.RowRowConverter;
+import org.apache.flink.table.types.CollectionDataType;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.FieldsDataType;
+import org.apache.flink.table.types.KeyValueDataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+
+public class FlinkRowToRowDataConverter {

Review Comment:
   It seems flink has utility class to convert Row to RowData, maybe we can use 
it. See RowRowConvertor



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/source/VeloxSourceBuilder.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.common.source;
+
+import org.apache.gluten.streaming.api.operators.GlutenStreamSource;
+import org.apache.gluten.table.runtime.operators.GlutenValuesSourceFunction;
+import org.apache.gluten.util.LogicalTypeConverter;
+import org.apache.gluten.util.PlanNodeIdGenerator;
+import org.apache.gluten.util.ReflectUtils;
+
+import io.github.zhztheplayer.velox4j.connector.VectorConnectorSplit;
+import io.github.zhztheplayer.velox4j.connector.VectorTableHandle;
+import io.github.zhztheplayer.velox4j.plan.StatefulPlanNode;
+import io.github.zhztheplayer.velox4j.plan.TableScanNode;
+
+import org.apache.flink.api.dag.Transformation;
+import 
org.apache.flink.streaming.api.transformations.LegacySourceTransformation;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+public class VeloxSourceBuilder {
+
+  public static Transformation<RowData> build(
+      Transformation<RowData> transformation, ScanTableSource scanTableSource) 
{
+    if (transformation instanceof LegacySourceTransformation) {
+      if 
(scanTableSource.getClass().getSimpleName().equals("TestValuesScanLookupTableSource"))
 {
+        return buildVectorSource(transformation, scanTableSource);
+      }
+    }
+    return transformation;
+  }
+
+  @SuppressWarnings({"rawtypes", "unchecked"})
+  private static Transformation<RowData> buildVectorSource(
+      Transformation<RowData> transformation, ScanTableSource tableSource) {
+    LegacySourceTransformation<RowData> sourceTransformation =
+        (LegacySourceTransformation<RowData>) transformation;
+    try {
+      Class<?> tableSourceClazz =
+          Class.forName(
+              
"org.apache.flink.table.planner.factories.TestValuesTableFactory$TestValuesScanTableSourceWithoutProjectionPushDown");
+      Map<Map<String, String>, Collection<Row>> data =
+          (Map) ReflectUtils.getObjectField(tableSourceClazz, tableSource, 
"data");
+      InternalTypeInfo<RowData> typeInfo =
+          (InternalTypeInfo<RowData>) sourceTransformation.getOutputType();
+      io.github.zhztheplayer.velox4j.type.RowType rowType =
+          (io.github.zhztheplayer.velox4j.type.RowType)
+              LogicalTypeConverter.toVLType(typeInfo.toLogicalType());
+      List<Row> values = new ArrayList<>();
+      for (Collection<Row> rows : data.values()) {
+        for (Row row : rows) {
+          Row projectedRow =
+              (Row)
+                  ReflectUtils.invokeObjectMethod(
+                      tableSourceClazz,
+                      tableSource,
+                      "projectRow",
+                      new Class<?>[] {Row.class},
+                      new Object[] {row});
+          values.add(projectedRow);
+        }
+      }
+      VectorTableHandle tableHandle =
+          new VectorTableHandle("connector-vector", "vector-table", rowType);
+      TableScanNode scanNode =
+          new TableScanNode(PlanNodeIdGenerator.newId(), rowType, tableHandle, 
List.of());
+      GlutenStreamSource op =
+          new GlutenStreamSource(
+              new GlutenValuesSourceFunction(

Review Comment:
   It need not to add a GlutenValuesSourceFunction, you can pass the values to 
TableScanNode



##########
gluten-flink/planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * 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.flink.table.planner.plan.nodes.exec.stream;
+
+import org.apache.flink.FlinkVersion;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
+import 
org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan;
+import 
org.apache.flink.table.planner.plan.nodes.exec.common.source.VeloxSourceBuilder;
+import 
org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec;
+import org.apache.flink.table.planner.utils.ShortcutUtils;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import 
org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
+
+import java.util.Collections;
+
+/**
+ * Stream {@link ExecNode} to read data from an external source defined by a 
{@link
+ * ScanTableSource}.
+ */
+@ExecNodeMetadata(
+    name = "stream-exec-table-source-scan",
+    version = 1,
+    producedTransformations = CommonExecTableSourceScan.SOURCE_TRANSFORMATION,
+    minPlanVersion = FlinkVersion.v1_15,
+    minStateVersion = FlinkVersion.v1_15)
+public class StreamExecTableSourceScan extends CommonExecTableSourceScan
+    implements StreamExecNode<RowData> {
+
+  public StreamExecTableSourceScan(
+      ReadableConfig tableConfig,
+      DynamicTableSourceSpec tableSourceSpec,
+      RowType outputType,
+      String description) {
+    this(
+        ExecNodeContext.newNodeId(),
+        ExecNodeContext.newContext(StreamExecTableSourceScan.class),
+        ExecNodeContext.newPersistedConfig(StreamExecTableSourceScan.class, 
tableConfig),
+        tableSourceSpec,
+        outputType,
+        description);
+  }
+
+  @JsonCreator
+  public StreamExecTableSourceScan(
+      @JsonProperty(FIELD_NAME_ID) int id,
+      @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+      @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig,
+      @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec 
tableSourceSpec,
+      @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
+      @JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
+    super(
+        id,
+        context,
+        persistedConfig,
+        tableSourceSpec,
+        Collections.emptyList(),
+        outputType,
+        description);
+  }
+
+  @Override
+  public Transformation<RowData> createInputFormatTransformation(
+      StreamExecutionEnvironment env,
+      InputFormat<RowData, ?> inputFormat,
+      InternalTypeInfo<RowData> outputTypeInfo,
+      String operatorName) {
+    // It's better to use StreamExecutionEnvironment.createInput()
+    // rather than addLegacySource() for streaming, because it take care of 
checkpoint.
+    return env.createInput(inputFormat, 
outputTypeInfo).name(operatorName).getTransformation();
+  }
+
+  @Override
+  protected Transformation<RowData> translateToPlanInternal(
+      PlannerBase planner, ExecNodeConfig config) {
+    final ScanTableSource tableSource =
+        getTableSourceSpec()
+            .getScanTableSource(
+                planner.getFlinkContext(), 
ShortcutUtils.unwrapTypeFactory(planner));
+    Transformation<RowData> sourceTransformation = 
super.translateToPlanInternal(planner, config);
+    return VeloxSourceBuilder.build(sourceTransformation, tableSource);

Review Comment:
   Add comments for gluten specified changes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to