> optionalOptions() {
+ return Collections.singleton(FlinkOptions.PATH);
}
}
diff --git a/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java b/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java
index 446aba0a75bd8..d9435132076c4 100644
--- a/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java
+++ b/hudi-flink/src/test/java/org/apache/hudi/utils/source/ContinuousFileSource.java
@@ -27,10 +27,12 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.source.DataStreamScanProvider;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
-import org.apache.flink.table.sources.StreamTableSource;
-import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import java.io.IOException;
@@ -53,7 +55,7 @@
*
* If all the data are flushed out, it waits for the next checkpoint to finish and tear down the source.
*/
-public class ContinuousFileSource implements StreamTableSource {
+public class ContinuousFileSource implements ScanTableSource {
private final TableSchema tableSchema;
private final Path path;
@@ -69,30 +71,46 @@ public ContinuousFileSource(
}
@Override
- public DataStream getDataStream(StreamExecutionEnvironment execEnv) {
- final RowType rowType = (RowType) this.tableSchema.toRowDataType().getLogicalType();
- JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
- rowType,
- new RowDataTypeInfo(rowType),
- false,
- true,
- TimestampFormat.ISO_8601);
-
- return execEnv.addSource(new BoundedSourceFunction(this.path, 2))
- .name("continuous_file_source")
- .setParallelism(1)
- .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
- new RowDataTypeInfo(rowType));
+ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) {
+ return new DataStreamScanProvider() {
+
+ @Override
+ public boolean isBounded() {
+ return false;
+ }
+
+ @Override
+ public DataStream produceDataStream(StreamExecutionEnvironment execEnv) {
+ final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+ JsonRowDataDeserializationSchema deserializationSchema = new JsonRowDataDeserializationSchema(
+ rowType,
+ InternalTypeInfo.of(rowType),
+ false,
+ true,
+ TimestampFormat.ISO_8601);
+
+ return execEnv.addSource(new BoundedSourceFunction(path, 2))
+ .name("continuous_file_source")
+ .setParallelism(1)
+ .map(record -> deserializationSchema.deserialize(record.getBytes(StandardCharsets.UTF_8)),
+ InternalTypeInfo.of(rowType));
+ }
+ };
+ }
+
+ @Override
+ public ChangelogMode getChangelogMode() {
+ return ChangelogMode.insertOnly();
}
@Override
- public TableSchema getTableSchema() {
- return this.tableSchema;
+ public DynamicTableSource copy() {
+ return new ContinuousFileSource(this.tableSchema, this.path, this.conf);
}
@Override
- public DataType getProducedDataType() {
- return this.tableSchema.toRowDataType().bridgedTo(RowData.class);
+ public String asSummaryString() {
+ return "ContinuousFileSource";
}
/**
diff --git a/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 5fec9b622e06b..19e43c4f81d82 100644
--- a/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -14,4 +14,5 @@
# See the License for the specific language governing permissions and
# limitations under the License.
+org.apache.hudi.utils.factory.ContinuousFileSourceFactory
org.apache.hudi.utils.factory.CollectSinkTableFactory
diff --git a/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory
deleted file mode 100644
index 87c3d990a1798..0000000000000
--- a/hudi-flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.TableFactory
+++ /dev/null
@@ -1,18 +0,0 @@
-# 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.
-
-org.apache.hudi.table.HoodieTableFactory
-org.apache.hudi.utils.factory.ContinuousFileSourceFactory
diff --git a/packaging/hudi-flink-bundle/pom.xml b/packaging/hudi-flink-bundle/pom.xml
index 29913937c3a74..7a30b74fb2f8b 100644
--- a/packaging/hudi-flink-bundle/pom.xml
+++ b/packaging/hudi-flink-bundle/pom.xml
@@ -33,6 +33,8 @@
${project.parent.basedir}
org.apache.hudi.
3.1.0
+
+ 1.11.1
@@ -89,8 +91,9 @@
org.apache.parquet:parquet-hadoop
org.apache.parquet:parquet-column
org.apache.parquet:parquet-common
- org.apache.parquet:parquet-format
+ org.apache.parquet:parquet-format-structures
org.apache.parquet:parquet-encoding
+ org.apache.parquet:parquet-jackson
org.apache.avro:avro
joda-time:joda-time
@@ -100,11 +103,6 @@
com.twitter:bijection-avro_${scala.binary.version}
com.twitter:bijection-core_${scala.binary.version}
- io.confluent:kafka-avro-serializer
- io.confluent:common-config
- io.confluent:common-utils
- io.confluent:kafka-schema-registry-client
- org.apache.kafka:kafka-clients
io.dropwizard.metrics:metrics-core
io.dropwizard.metrics:metrics-graphite
io.prometheus:simpleclient
@@ -113,16 +111,12 @@
io.prometheus:simpleclient_pushgateway
io.prometheus:simpleclient_common
com.yammer.metrics:metrics-core
- org.apache.kafka:kafka_${scala.binary.version}
- com.101tec:zkclient
org.eclipse.jetty:*
org.eclipse.jetty.websocket:*
javax.servlet:javax.servlet-api
- org.apache.flink:flink-connector-kafka_${scala.binary.version}
- org.apache.flink:flink-connector-kafka-base_${scala.binary.version}
org.apache.flink:flink-hadoop-compatibility_${scala.binary.version}
org.apache.flink:flink-avro
org.apache.flink:flink-json
@@ -133,14 +127,18 @@
org.apache.hive:hive-metastore
org.apache.hive:hive-jdbc
- org.apache.hbase:hbase-client
org.apache.hbase:hbase-common
- org.apache.hbase:hbase-protocol
- org.apache.hbase:hbase-server
- org.apache.htrace:htrace-core
+
+ org.apache.avro.
+ ${flink.bundle.hive.shade.prefix}org.apache.avro.
+
+
+ org.apache.parquet.
+ ${flink.bundle.hive.shade.prefix}org.apache.parquet.
+
com.yammer.metrics.
org.apache.hudi.com.yammer.metrics.
@@ -198,8 +196,6 @@
META-INF/*.DSA
META-INF/*.RSA
META-INF/services/javax.*
-
- META-INF/services/org.apache.flink.table.factories.Factory
@@ -258,17 +254,6 @@
-
- org.apache.flink
- flink-connector-kafka_${scala.binary.version}
- compile
-
-
- org.apache.flink
- flink-connector-kafka-base_${scala.binary.version}
- ${flink.version}
- compile
-
org.apache.flink
flink-hadoop-compatibility_${scala.binary.version}
@@ -295,9 +280,62 @@
+
+ org.apache.parquet
+ parquet-hadoop
+ ${parquet.version}
+ compile
+
+
+ org.xerial.snappy
+ snappy-java
+
+
+
org.apache.parquet
parquet-avro
+ ${parquet.version}
+ compile
+
+
+ org.apache.hadoop
+ hadoop-client
+
+
+ it.unimi.dsi
+ fastutil
+
+
+
+
+ org.apache.parquet
+ parquet-column
+ ${parquet.version}
+ compile
+
+
+ org.apache.parquet
+ parquet-common
+ ${parquet.version}
+ compile
+
+
+ org.apache.parquet
+ parquet-encoding
+ ${parquet.version}
+ compile
+
+
+ org.apache.parquet
+ parquet-format-structures
+ ${parquet.version}
+ compile
+
+
+ org.apache.parquet
+ parquet-jackson
+ ${parquet.version}
compile
@@ -391,52 +429,11 @@
compile
-
- org.apache.htrace
- htrace-core
- ${htrace.version}
- compile
-
-
org.apache.hbase
hbase-common
${hbase.version}
-
- org.apache.hbase
- hbase-server
- ${hbase.version}
- compile
-
-
- javax.servlet
- *
-
-
- org.codehaus.jackson
- *
-
-
- org.mortbay.jetty
- *
-
-
- tomcat
- *
-
-
-
-
- org.apache.hbase
- hbase-client
- ${hbase.version}
-
-
- org.apache.hbase
- hbase-protocol
- ${hbase.version}
-
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index cfd924145478a..4c950feb1c578 100644
--- a/pom.xml
+++ b/pom.xml
@@ -107,7 +107,7 @@
4.4.1
${spark2.version}
${spark2bundle.version}
- 1.11.2
+ 1.12.2
2.4.4
3.0.0
diff --git a/style/checkstyle-suppressions.xml b/style/checkstyle-suppressions.xml
index 401a3eadf7a7a..c2e63db143b13 100644
--- a/style/checkstyle-suppressions.xml
+++ b/style/checkstyle-suppressions.xml
@@ -28,5 +28,6 @@
+