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



##########
File path: flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.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.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+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() {
+  }
+
+  public static Builder forRow(DataStream<Row> input) {
+    return new Builder().forRow(input);
+  }
+
+  public static Builder forRowData(DataStream<RowData> input) {
+    return new Builder().forRowData(input);
+  }
+
+  public static class Builder {
+    private DataStream<Row> rowInput = null;
+    private DataStream<RowData> rowDataInput = null;
+    private TableLoader tableLoader;
+    private Configuration hadoopConf;
+    private Table table;
+    private TableSchema tableSchema;
+
+    private Builder forRow(DataStream<Row> newRowInput) {
+      this.rowInput = newRowInput;
+      return this;
+    }
+
+    private Builder forRowData(DataStream<RowData> newRowDataInput) {
+      this.rowDataInput = newRowDataInput;
+      return this;
+    }
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    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;
+    }
+
+    private DataStream<RowData> convert() {
+      Preconditions.checkArgument(rowInput != null, "The DataStream<Row> to 
convert shouldn't be null");

Review comment:
       The preconditions in build ensure this is never the case. I'm fine 
keeping the check (up to you) but it seems odd to have it in a private method. 
Also, this relies on the check that `table` is not null when `tableSchema` is 
null.




----------------------------------------------------------------
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:
[email protected]



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

Reply via email to