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/13 09:37:31 UTC

[GitHub] [iceberg] openinx commented on a change in pull request #1293: Flink: Implement Flink InputFormat and integrate it to FlinkCatalog

openinx commented on a change in pull request #1293:
URL: https://github.com/apache/iceberg/pull/1293#discussion_r469806393



##########
File path: flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.io.Serializable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hive.HiveCatalog;
+
+/**
+ * Serializable loader to load an Iceberg {@link Catalog}.
+ * Flink needs to get {@link Table} objects in the cluster (for example, to get splits), not just on the client side.
+ * So we need an Iceberg catalog loader to get the {@link Catalog} and get the {@link Table} object.
+ */
+public interface CatalogLoader extends Serializable {

Review comment:
       Seems we could move this class to hive modules ? 

##########
File path: flink/src/test/java/org/apache/iceberg/flink/data/RandomData.java
##########
@@ -88,6 +93,34 @@ public Row next() {
     };
   }
 
+  private static Iterable<RowData> generateRowData(Schema schema, int numRecords,

Review comment:
       We will remove this `RandomData` class in the future (Once RowData parquet reader & writer is ready),   we've already had the `RandomRowData` to generate `Iterable<RowData>`, you may want to rebase master branch now.  (so we don't have to define a `generateRowData` here now).
   It will generate `Record` firstly, then convert them into `RowData`, in this way we could assert the RowData with Record,  it can be more rigorous to determine the correctness of the data.

##########
File path: data/src/test/java/org/apache/iceberg/TestAppendHelper.java
##########
@@ -0,0 +1,128 @@
+/*
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.orc.GenericOrcWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.junit.Assert;
+import org.junit.rules.TemporaryFolder;
+
+public class TestAppendHelper {
+
+  private final Table table;
+  private final FileFormat fileFormat;
+  private final TemporaryFolder tmp;
+
+  public TestAppendHelper(Table table, FileFormat fileFormat, TemporaryFolder tmp) {
+    this.table = table;
+    this.fileFormat = fileFormat;
+    this.tmp = tmp;
+  }
+
+  public void appendToTable(DataFile... dataFiles) {
+    Preconditions.checkNotNull(table, "table not set");
+
+    AppendFiles append = table.newAppend();
+
+    for (DataFile dataFile : dataFiles) {
+      append = append.appendFile(dataFile);
+    }
+
+    append.commit();
+  }
+
+  public void appendToTable(List<Record> records) throws IOException {
+    appendToTable(null, records);
+  }
+
+  public void appendToTable(StructLike partition, List<Record> records) throws IOException {
+    appendToTable(writeFile(partition, records));
+  }
+
+  public DataFile writeFile(StructLike partition, List<Record> records) throws IOException {
+    Preconditions.checkNotNull(table, "table not set");
+    return writeFile(table, partition, records, fileFormat, tmp.newFile());
+  }
+
+  public static DataFile writeFile(
+      Table table, StructLike partition, List<Record> records, FileFormat fileFormat,
+      File file) throws IOException {
+    Assert.assertTrue(file.delete());
+
+    FileAppender<Record> appender;
+
+    switch (fileFormat) {
+      case AVRO:
+        appender = Avro.write(Files.localOutput(file))
+            .schema(table.schema())
+            .createWriterFunc(DataWriter::create)
+            .named(fileFormat.name())
+            .build();
+        break;
+
+      case PARQUET:
+        appender = Parquet.write(Files.localOutput(file))
+            .schema(table.schema())
+            .createWriterFunc(GenericParquetWriter::buildWriter)
+            .named(fileFormat.name())
+            .build();
+        break;
+
+      case ORC:
+        appender = ORC.write(Files.localOutput(file))
+            .schema(table.schema())
+            .createWriterFunc(GenericOrcWriter::buildWriter)
+            .build();
+        break;
+
+      default:
+        throw new UnsupportedOperationException("Cannot write format: " + fileFormat);
+    }
+
+    try {
+      appender.addAll(records);
+    } finally {
+      appender.close();
+    }

Review comment:
       Seems we could abstract those appender building into a separate method,  I saw lots of unit tests depend on this common logics, could be a separate issue to do this.




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