Skip to content
Merged
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
2 changes: 1 addition & 1 deletion .github/workflows/flink.yml
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ jobs:
source /opt/rh/gcc-toolset-11/enable
sudo dnf install -y patchelf
git clone -b gluten-0530 https://github.com/bigo-sg/velox4j.git
cd velox4j && git reset --hard 1cdeb1a8384967499919e655d55a66f2daa9d55c
cd velox4j && git reset --hard ea2ca5755ae91a8703717a85b77f9eb1620899de
git apply $GITHUB_WORKSPACE/gluten-flink/patches/fix-velox4j.patch
mvn clean install -DskipTests -Dgpg.skip -Dspotless.skip=true
cd ..
Expand Down
2 changes: 1 addition & 1 deletion gluten-flink/docs/Flink.md
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ As some features have not been committed to upstream, you have to use the follow
## fetch velox4j code
git clone -b gluten-0530 https://github.com/bigo-sg/velox4j.git
cd velox4j
git reset --hard 1cdeb1a8384967499919e655d55a66f2daa9d55c
git reset --hard ea2ca5755ae91a8703717a85b77f9eb1620899de
mvn clean install -DskipTests -Dgpg.skip -Dspotless.skip=true
```
**Get gluten**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import org.apache.gluten.table.runtime.operators.GlutenOneInputOperator;
import org.apache.gluten.util.LogicalTypeConverter;
import org.apache.gluten.util.PlanNodeIdGenerator;
import org.apache.gluten.velox.VeloxSinkBuilder;

import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.api.dag.Transformation;
Expand Down Expand Up @@ -96,7 +97,6 @@
*/
public abstract class CommonExecSink extends ExecNodeBase<Object>
implements MultipleTransformationTranslator<Object> {

public static final String CONSTRAINT_VALIDATOR_TRANSFORMATION = "constraint-validator";
public static final String PARTITIONER_TRANSFORMATION = "partitioner";
public static final String UPSERT_MATERIALIZE_TRANSFORMATION = "upsert-materialize";
Expand Down Expand Up @@ -466,8 +466,12 @@ 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);
// --- Begin Gluten-specific code changes ---
Transformation sinkTransformation =
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add // --- Begin Gluten-specific code changes --- before and after your changes

createSinkFunctionTransformation(
sinkFunction, env, inputTransform, rowtimeFieldIndex, sinkMeta, sinkParallelism);
return VeloxSinkBuilder.build(env.getConfiguration(), sinkTransformation);
// --- End Gluten-specific code changes ---
} else if (runtimeProvider instanceof OutputFormatProvider) {
OutputFormat<RowData> outputFormat =
((OutputFormatProvider) runtimeProvider).createOutputFormat();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;

import org.apache.gluten.streaming.api.operators.GlutenOperator;
import org.apache.gluten.streaming.runtime.partitioner.GlutenKeyGroupStreamPartitioner;
import org.apache.gluten.table.runtime.keyselector.GlutenKeySelector;
import org.apache.gluten.table.runtime.operators.GlutenVectorOneInputOperator;
import org.apache.gluten.util.LogicalTypeConverter;
Expand All @@ -39,7 +40,6 @@
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
import org.apache.flink.streaming.runtime.partitioner.GlutenKeyGroupStreamPartitioner;
import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.table.api.TableException;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.gluten.velox.VeloxSourceBuilder;

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.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) {
// --- Begin Gluten-specific code changes ---
final ScanTableSource tableSource =
getTableSourceSpec()
.getScanTableSource(
planner.getFlinkContext(), ShortcutUtils.unwrapTypeFactory(planner));
Transformation<RowData> sourceTransformation = super.translateToPlanInternal(planner, config);
return VeloxSourceBuilder.build(sourceTransformation, tableSource);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add comments for gluten specified changes.

// --- End Gluten-specific code changes ---
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,112 @@
/*
* 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.velox;

import org.apache.gluten.streaming.api.operators.GlutenOneInputOperatorFactory;
import org.apache.gluten.table.runtime.operators.GlutenVectorOneInputOperator;
import org.apache.gluten.util.LogicalTypeConverter;
import org.apache.gluten.util.PlanNodeIdGenerator;

import io.github.zhztheplayer.velox4j.connector.CommitStrategy;
import io.github.zhztheplayer.velox4j.connector.PrintTableHandle;
import io.github.zhztheplayer.velox4j.plan.EmptyNode;
import io.github.zhztheplayer.velox4j.plan.StatefulPlanNode;
import io.github.zhztheplayer.velox4j.plan.TableWriteNode;
import io.github.zhztheplayer.velox4j.type.BigIntType;
import io.github.zhztheplayer.velox4j.type.RowType;

import org.apache.flink.api.dag.Transformation;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.CoreOptions;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
import org.apache.flink.streaming.api.transformations.LegacySinkTransformation;
import org.apache.flink.table.runtime.operators.sink.SinkOperator;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.util.FlinkRuntimeException;

import java.util.List;
import java.util.Map;

public class VeloxSinkBuilder {

public static Transformation build(ReadableConfig config, Transformation transformation) {
if (transformation instanceof LegacySinkTransformation) {
SimpleOperatorFactory operatorFactory =
(SimpleOperatorFactory) ((LegacySinkTransformation) transformation).getOperatorFactory();
OneInputStreamOperator sinkOp = (OneInputStreamOperator) operatorFactory.getOperator();
if (sinkOp instanceof SinkOperator
&& ((SinkOperator) sinkOp)
.getUserFunction()
.getClass()
.getSimpleName()
.equals("RowDataPrintFunction")) {
return buildPrintSink(config, (LegacySinkTransformation) transformation);
}
}
return transformation;
}

private static LegacySinkTransformation buildPrintSink(
ReadableConfig config, LegacySinkTransformation transformation) {
Transformation inputTrans = (Transformation) transformation.getInputs().get(0);
InternalTypeInfo inputTypeInfo = (InternalTypeInfo) inputTrans.getOutputType();
String logDir = config.get(CoreOptions.FLINK_LOG_DIR);
String printPath;
if (logDir != null) {
printPath = String.format("file://%s/%s", logDir, "taskmanager.out");
} else {
String flinkHomeDir = System.getenv(ConfigConstants.ENV_FLINK_HOME_DIR);
if (flinkHomeDir == null) {
String flinkConfDir = System.getenv(ConfigConstants.ENV_FLINK_CONF_DIR);
if (flinkConfDir == null) {
throw new FlinkRuntimeException(
"Can not get flink home directory, please set FLINK_HOME.");
}
printPath = String.format("file://%s/../log/%s", flinkConfDir, "taskmanager.out");
} else {
printPath = String.format("file://%s/log/%s", flinkHomeDir, "taskmanager.out");
}
}
RowType inputColumns = (RowType) LogicalTypeConverter.toVLType(inputTypeInfo.toLogicalType());
RowType ignore = new RowType(List.of("num"), List.of(new BigIntType()));
PrintTableHandle tableHandle = new PrintTableHandle("print-table", inputColumns, printPath);
TableWriteNode tableWriteNode =
new TableWriteNode(
PlanNodeIdGenerator.newId(),
inputColumns,
inputColumns.getNames(),
null,
"connector-print",
tableHandle,
false,
ignore,
CommitStrategy.NO_COMMIT,
List.of(new EmptyNode(inputColumns)));
return new LegacySinkTransformation(
inputTrans,
transformation.getName(),
new GlutenOneInputOperatorFactory(
new GlutenVectorOneInputOperator(
new StatefulPlanNode(tableWriteNode.getId(), tableWriteNode),
PlanNodeIdGenerator.newId(),
inputColumns,
Map.of(tableWriteNode.getId(), ignore))),
transformation.getParallelism());
}
}
Loading