-
Notifications
You must be signed in to change notification settings - Fork 3k
Flink: Integrate Flink reader to SQL #1509
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,111 @@ | ||
| /* | ||
| * 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.iceberg.flink; | ||
|
|
||
| import java.util.Arrays; | ||
| import java.util.Map; | ||
| import org.apache.flink.streaming.api.datastream.DataStream; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.table.api.TableSchema; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.sources.FilterableTableSource; | ||
| import org.apache.flink.table.sources.LimitableTableSource; | ||
| import org.apache.flink.table.sources.ProjectableTableSource; | ||
| import org.apache.flink.table.sources.StreamTableSource; | ||
| import org.apache.flink.table.sources.TableSource; | ||
| import org.apache.flink.table.types.DataType; | ||
| import org.apache.flink.table.utils.TableConnectorUtils; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.iceberg.flink.source.FlinkSource; | ||
|
|
||
| /** | ||
| * Flink Iceberg table source. | ||
| * TODO: Implement {@link FilterableTableSource} and {@link LimitableTableSource}. | ||
| */ | ||
| public class IcebergTableSource implements StreamTableSource<RowData>, ProjectableTableSource<RowData> { | ||
|
|
||
| private final TableLoader loader; | ||
| private final Configuration hadoopConf; | ||
| private final TableSchema schema; | ||
| private final Map<String, String> options; | ||
| private final int[] projectedFields; | ||
|
|
||
| public IcebergTableSource(TableLoader loader, Configuration hadoopConf, TableSchema schema, | ||
| Map<String, String> options) { | ||
| this(loader, hadoopConf, schema, options, null); | ||
| } | ||
|
|
||
| private IcebergTableSource(TableLoader loader, Configuration hadoopConf, TableSchema schema, | ||
| Map<String, String> options, int[] projectedFields) { | ||
| this.loader = loader; | ||
| this.hadoopConf = hadoopConf; | ||
| this.schema = schema; | ||
| this.options = options; | ||
| this.projectedFields = projectedFields; | ||
| } | ||
|
|
||
| @Override | ||
| public boolean isBounded() { | ||
| return FlinkSource.isBounded(options); | ||
| } | ||
|
|
||
| @Override | ||
| public TableSource<RowData> projectFields(int[] fields) { | ||
| return new IcebergTableSource(loader, hadoopConf, schema, options, fields); | ||
| } | ||
|
|
||
| @Override | ||
| public DataStream<RowData> getDataStream(StreamExecutionEnvironment execEnv) { | ||
| return FlinkSource.forRowData().env(execEnv).tableLoader(loader).hadoopConf(hadoopConf) | ||
| .project(getProjectedSchema()).properties(options).build(); | ||
| } | ||
|
|
||
| @Override | ||
| public TableSchema getTableSchema() { | ||
| return schema; | ||
| } | ||
|
|
||
| @Override | ||
| public DataType getProducedDataType() { | ||
| return getProjectedSchema().toRowDataType().bridgedTo(RowData.class); | ||
| } | ||
|
|
||
| private TableSchema getProjectedSchema() { | ||
| TableSchema fullSchema = getTableSchema(); | ||
| if (projectedFields == null) { | ||
| return fullSchema; | ||
| } else { | ||
| String[] fullNames = fullSchema.getFieldNames(); | ||
| DataType[] fullTypes = fullSchema.getFieldDataTypes(); | ||
| return TableSchema.builder().fields( | ||
| Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), | ||
| Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)).build(); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public String explainSource() { | ||
| String explain = "Iceberg table: " + loader.toString(); | ||
| if (projectedFields != null) { | ||
| explain += ", ProjectedFields: " + Arrays.toString(projectedFields); | ||
| } | ||
| return TableConnectorUtils.generateRuntimeName(getClass(), getTableSchema().getFieldNames()) + explain; | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,6 +22,7 @@ | |
| import java.io.IOException; | ||
| import java.io.UncheckedIOException; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import org.apache.flink.streaming.api.datastream.DataStream; | ||
| import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
| import org.apache.flink.table.api.TableSchema; | ||
|
|
@@ -47,11 +48,10 @@ private FlinkSource() { | |
|
|
||
| /** | ||
| * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}. | ||
| * See more options in {@link ScanOptions}. | ||
| * See more options in {@link ScanContext}. | ||
| * <p> | ||
| * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and | ||
| * read records incrementally. | ||
| * The Bounded and Unbounded depends on the {@link Builder#options(ScanOptions)}: | ||
| * <ul> | ||
| * <li>Without startSnapshotId: Bounded</li> | ||
| * <li>With startSnapshotId and with endSnapshotId: Bounded</li> | ||
|
|
@@ -72,10 +72,9 @@ public static class Builder { | |
| private StreamExecutionEnvironment env; | ||
| private Table table; | ||
| private TableLoader tableLoader; | ||
| private TableSchema projectedSchema; | ||
| private ScanOptions options = ScanOptions.builder().build(); | ||
| private List<Expression> filterExpressions; | ||
| private Configuration hadoopConf; | ||
| private TableSchema projectedSchema; | ||
| private ScanContext context = new ScanContext(); | ||
|
|
||
| private RowDataTypeInfo rowTypeInfo; | ||
|
|
||
|
|
@@ -89,8 +88,18 @@ public Builder table(Table newTable) { | |
| return this; | ||
| } | ||
|
|
||
| public Builder filters(List<Expression> newFilters) { | ||
| this.filterExpressions = newFilters; | ||
| public Builder hadoopConf(Configuration newConf) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We try to avoid using Hadoop classes in Iceberg APIs because they are hard to remove. Injecting a Hadoop The catalog creates tables and tables are associated with a MR also has a Hadoop
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 for avoiding using Hadoop
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The reason why Hadoop conf needs to be passed now is because:
Maybe we can pass this chain with an Iceberg object (
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Or maybe this could use the approach from the public static Configuration clusterHadoopConf() {
return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration());
}Using
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Using |
||
| this.hadoopConf = newConf; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder env(StreamExecutionEnvironment newEnv) { | ||
| this.env = newEnv; | ||
| return this; | ||
| } | ||
|
|
||
| public Builder filters(List<Expression> filters) { | ||
| this.context = context.filterRows(filters); | ||
| return this; | ||
| } | ||
|
|
||
|
|
@@ -99,18 +108,53 @@ public Builder project(TableSchema schema) { | |
| return this; | ||
| } | ||
|
|
||
| public Builder options(ScanOptions newOptions) { | ||
| this.options = newOptions; | ||
| public Builder properties(Map<String, String> properties) { | ||
| this.context = context.fromProperties(properties); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder hadoopConf(Configuration newConf) { | ||
| this.hadoopConf = newConf; | ||
| public Builder caseSensitive(boolean caseSensitive) { | ||
| this.context = context.setCaseSensitive(caseSensitive); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder env(StreamExecutionEnvironment newEnv) { | ||
| this.env = newEnv; | ||
| public Builder snapshotId(Long snapshotId) { | ||
| this.context = context.useSnapshotId(snapshotId); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder startSnapshotId(Long startSnapshotId) { | ||
| this.context = context.startSnapshotId(startSnapshotId); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder endSnapshotId(Long endSnapshotId) { | ||
| this.context = context.endSnapshotId(endSnapshotId); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder asOfTimestamp(Long asOfTimestamp) { | ||
| this.context = context.asOfTimestamp(asOfTimestamp); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder splitSize(Long splitSize) { | ||
| this.context = context.splitSize(splitSize); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder splitLookback(Integer splitLookback) { | ||
| this.context = context.splitLookback(splitLookback); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder splitOpenFileCost(Long splitOpenFileCost) { | ||
| this.context = context.splitOpenFileCost(splitOpenFileCost); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder nameMapping(String nameMapping) { | ||
| this.context = context.nameMapping(nameMapping); | ||
| return this; | ||
| } | ||
|
|
||
|
|
@@ -144,23 +188,28 @@ public FlinkInputFormat buildFormat() { | |
| FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema)) : | ||
| projectedSchema).toRowDataType().getLogicalType()); | ||
|
|
||
| Schema expectedSchema = icebergSchema; | ||
| if (projectedSchema != null) { | ||
| expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedSchema); | ||
| } | ||
| context = context.project(projectedSchema == null ? icebergSchema : | ||
| FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); | ||
|
|
||
| return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options, | ||
| new SerializableConfiguration(hadoopConf)); | ||
| return new FlinkInputFormat(tableLoader, new SerializableConfiguration(hadoopConf), io, encryption, context); | ||
| } | ||
|
|
||
| public DataStream<RowData> build() { | ||
| Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); | ||
| FlinkInputFormat format = buildFormat(); | ||
| if (options.startSnapshotId() != null && options.endSnapshotId() == null) { | ||
| throw new UnsupportedOperationException("The Unbounded mode is not supported yet"); | ||
| } else { | ||
| if (isBounded(context)) { | ||
| return env.createInput(format, rowTypeInfo); | ||
| } else { | ||
| throw new UnsupportedOperationException("The Unbounded mode is not supported yet"); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private static boolean isBounded(ScanContext context) { | ||
| return context.startSnapshotId() == null || context.endSnapshotId() != null; | ||
| } | ||
|
|
||
| public static boolean isBounded(Map<String, String> properties) { | ||
| return isBounded(new ScanContext().fromProperties(properties)); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.