You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/28 17:44:30 UTC

[GitHub] [iceberg] stevenzwu commented on a change in pull request #1185: Flink: Add the iceberg files committer to collect data files and commit to iceberg table.

stevenzwu commented on a change in pull request #1185:
URL: https://github.com/apache/iceberg/pull/1185#discussion_r479448928



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.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.iceberg.flink.sink;
+
+import java.util.Locale;
+import java.util.Map;
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.util.DataFormatConverters;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES;
+import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT;
+
+public class FlinkSink {
+
+  private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName();
+  private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName();
+
+  private FlinkSink() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from generic input data stream into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a mapper function and a
+   * {@link TypeInformation} to convert those generic records to a RowData DataStream.
+   *
+   * @param input      the generic source input data stream.
+   * @param mapper     function to convert the generic data to {@link RowData}
+   * @param outputType to define the {@link TypeInformation} for the input data.
+   * @param <T>        the data type of records.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static <T> Builder builderFor(DataStream<T> input,
+                                       MapFunction<T, RowData> mapper,
+                                       TypeInformation<RowData> outputType) {
+    DataStream<RowData> dataStream = input.map(mapper, outputType);
+    return forRowData(dataStream);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into iceberg table. We use
+   * {@link RowData} inside the sink connector, so users need to provide a {@link TableSchema} for builder to convert
+   * those {@link Row}s to a {@link RowData} DataStream.
+   *
+   * @param input       the source input data stream with {@link Row}s.
+   * @param tableSchema defines the {@link TypeInformation} for input data.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRow(DataStream<Row> input, TableSchema tableSchema) {
+    RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType();
+    DataType[] fieldDataTypes = tableSchema.getFieldDataTypes();
+
+    DataFormatConverters.RowConverter rowConverter = new DataFormatConverters.RowConverter(fieldDataTypes);
+    return builderFor(input, rowConverter::toInternal, RowDataTypeInfo.of(rowType))
+        .tableSchema(tableSchema);
+  }
+
+  /**
+   * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s into iceberg table.
+   *
+   * @param input the source input data stream with {@link RowData}s.
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder() {
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    /**
+     * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} which will write all
+     * the records into {@link DataFile}s and emit them to downstream operator. Providing a table would avoid so many
+     * table loading from each separate task.
+     *
+     * @param newTable the loaded iceberg table instance.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    /**
+     * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need this loader because
+     * {@link Table} is not serializable and could not just use the loaded table from Builder#table in the remote task
+     * manager.
+     *
+     * @param newTableLoader to load iceberg table inside tasks.
+     * @return {@link Builder} to connect the iceberg table.
+     */
+    public Builder tableLoader(TableLoader newTableLoader) {
+      this.tableLoader = newTableLoader;
+      return this;
+    }
+
+    public Builder hadoopConf(Configuration newHadoopConf) {
+      this.hadoopConf = newHadoopConf;
+      return this;
+    }
+
+    public Builder tableSchema(TableSchema newTableSchema) {
+      this.tableSchema = newTableSchema;
+      return this;
+    }
+
+    @SuppressWarnings("unchecked")
+    public DataStreamSink<RowData> build() {
+      Preconditions.checkArgument(rowDataInput != null,
+          "Please use forRowData() to initialize the input DataStream.");
+      Preconditions.checkNotNull(table, "Table shouldn't be null");
+      Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null");
+      Preconditions.checkNotNull(hadoopConf, "Hadoop configuration shouldn't be null");
+
+      IcebergStreamWriter<RowData> streamWriter = createStreamWriter(table, tableSchema);
+      IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, hadoopConf);
+
+      DataStream<Void> returnStream = rowDataInput
+          .transform(ICEBERG_STREAM_WRITER_NAME, TypeInformation.of(DataFile.class), streamWriter)

Review comment:
       there can be multiple Iceberg sinks in the same job. we probably should add the table identifier string suffix to make operator name and id unique.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org