Fokko commented on a change in pull request #1972:
URL: https://github.com/apache/iceberg/pull/1972#discussion_r549489600



##########
File path: beam/src/main/java/org/apache/iceberg/beam/IcebergIO.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.beam;
+
+import org.apache.beam.sdk.io.WriteFilesResult;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.values.KV;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+import java.util.LinkedList;
+import java.util.List;
+
+
+public class IcebergIO {
+
+  private IcebergIO() {
+  }
+
+  static void write(TableIdentifier table,
+                    Schema schema,
+                    String hiveMetastoreUrl,
+                    WriteFilesResult<Void> resultFiles) {
+    resultFiles.getPerDestinationOutputFilenames().apply(
+        Combine.globally(new FileCombiner(table, schema, 
hiveMetastoreUrl)).withoutDefaults()
+    );
+  }
+
+  private static class FileCombiner extends Combine.CombineFn<KV<Void, 
String>, List<String>, Snapshot> {
+    private final TableIdentifier tableIdentifier;
+    private final Schema schema;
+    private final String hiveMetastoreUrl;
+
+    FileCombiner(TableIdentifier table, Schema schema, String 
hiveMetastoreUrl) {
+      this.tableIdentifier = table;
+      this.schema = schema;
+      this.hiveMetastoreUrl = hiveMetastoreUrl;
+    }
+
+    @Override
+    public List<String> createAccumulator() {
+      return new LinkedList<>();
+    }
+
+    @Override
+    public List<String> addInput(List<String> mutableAccumulator, KV<Void, 
String> input) {
+      mutableAccumulator.add(input.getValue());
+      return mutableAccumulator;
+    }
+
+    @Override
+    public List<String> mergeAccumulators(Iterable<List<String>> accumulators) 
{
+      return ImmutableList.copyOf(Iterables.concat(accumulators));
+    }
+
+    @Override
+    public Snapshot extractOutput(List<String> accumulator) {
+      try (HiveCatalog catalog = new HiveCatalog(
+          HiveCatalog.DEFAULT_NAME,
+          this.hiveMetastoreUrl,
+          1,
+          new Configuration()
+      )) {
+        Table table;
+        try {
+          table = catalog.loadTable(this.tableIdentifier);
+        } catch (NoSuchTableException e) {
+          // If it doesn't exist, we just create the table
+          table = catalog.createTable(this.tableIdentifier, schema);
+        }
+
+        // In case the schema has changed
+        if (table.schema() != this.schema) {
+          table.updateSchema().unionByNameWith(this.schema).commit();

Review comment:
       Good point, and fully agree. I'll remove this one for now, this should 
be handled outside of the sink.

##########
File path: site/docs/beam.md
##########
@@ -0,0 +1,39 @@
+<!--
+ - 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.
+ -->
+
+# Apache Beam
+
+!!! Warning
+    The Beam API is experimental and in early alpha stage
+
+## Write records
+
+When sinking Avro files to a distributed file system using `FileIO`, we can 
catch rely the filenames downstream. We append these filenames to an Iceberg 
table incrementally.
+
+```java
+final String hiveMetastoreUrl = "thrift://localhost:9083/default";
+final FileIO.Write<Void, GenericRecord> avroFileIO = 
FileIO.<GenericRecord>write()
+        .via(AvroIO.sink(avroSchema))
+        .to("gs://.../../")
+        .withSuffix(".avro");

Review comment:
       What is the implication of doing this the other way around? In the 
current setup we have the Avro schema, and distill the Iceberg schema from it.

##########
File path: beam/src/test/java/org/apache/iceberg/beam/IcebergIOTest.java
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed 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.beam;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.AvroIO;
+import org.apache.beam.sdk.io.FileIO;
+import org.apache.beam.sdk.io.WriteFilesResult;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.testing.TestStream;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.windowing.FixedWindows;
+import org.apache.beam.sdk.transforms.windowing.Window;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+public class IcebergIOTest {
+    private static final List<String> SENTENCES =
+        Arrays.asList(
+            "Beam window 1 1",
+            "Beam window 1 2",
+            "Beam window 1 3",
+            "Beam window 1 4",
+            "Beam window 2 1",
+            "Beam window 2 2");
+    private static final Instant START_TIME = new Instant(0);
+
+    private static final List<String> FIRST_WIN_WORDS = SENTENCES.subList(0, 
4);
+    private static final List<String> SECOND_WIN_WORDS = SENTENCES.subList(4, 
6);
+    private static final Duration WINDOW_DURATION = 
Duration.standardMinutes(1);
+
+    @Rule
+    public final transient TestPipeline pipeline = TestPipeline.create();
+    final String hiveMetastoreUrl = "thrift://localhost:9083/default";
+
+    private static final PipelineOptions options = 
TestPipeline.testingPipelineOptions();
+
+    private static final String stringSchema = "{\n" +

Review comment:
       Yes, I've started by implementing Avro files, because this is our use 
case. But with the Beam API, Parquet and ORC are also supported.
   
   For the actual interface, we only use the Iceberg schema. The `IcebergIO` 
only accepts `org.apache.iceberg.Schema`. For the test we convert the Avro 
schema to iceberg using `AvroSchemaUtil.toIceberg(avroSchema);`.
   
   
![image](https://user-images.githubusercontent.com/1134248/103349632-a5634780-4a9d-11eb-9e96-9c8aed910fab.png)
   
   translates into:
   
   
![image](https://user-images.githubusercontent.com/1134248/103349642-ac8a5580-4a9d-11eb-9635-e0a99885933b.png)
   
   
   I need to clean the test up a bit.

##########
File path: beam/src/main/java/org/apache/iceberg/beam/IcebergIO.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.beam;
+
+import org.apache.beam.sdk.io.WriteFilesResult;
+import org.apache.beam.sdk.transforms.Combine;
+import org.apache.beam.sdk.values.KV;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.exceptions.NoSuchTableException;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+
+import java.util.LinkedList;
+import java.util.List;
+
+
+public class IcebergIO {
+
+  private IcebergIO() {
+  }
+
+  static void write(TableIdentifier table,
+                    Schema schema,
+                    String hiveMetastoreUrl,
+                    WriteFilesResult<Void> resultFiles) {
+    resultFiles.getPerDestinationOutputFilenames().apply(
+        Combine.globally(new FileCombiner(table, schema, 
hiveMetastoreUrl)).withoutDefaults()
+    );
+  }
+
+  private static class FileCombiner extends Combine.CombineFn<KV<Void, 
String>, List<String>, Snapshot> {
+    private final TableIdentifier tableIdentifier;
+    private final Schema schema;
+    private final String hiveMetastoreUrl;
+
+    FileCombiner(TableIdentifier table, Schema schema, String 
hiveMetastoreUrl) {
+      this.tableIdentifier = table;
+      this.schema = schema;
+      this.hiveMetastoreUrl = hiveMetastoreUrl;
+    }
+
+    @Override
+    public List<String> createAccumulator() {
+      return new LinkedList<>();
+    }
+
+    @Override
+    public List<String> addInput(List<String> mutableAccumulator, KV<Void, 
String> input) {
+      mutableAccumulator.add(input.getValue());
+      return mutableAccumulator;
+    }
+
+    @Override
+    public List<String> mergeAccumulators(Iterable<List<String>> accumulators) 
{
+      return ImmutableList.copyOf(Iterables.concat(accumulators));
+    }
+
+    @Override
+    public Snapshot extractOutput(List<String> accumulator) {
+      try (HiveCatalog catalog = new HiveCatalog(
+          HiveCatalog.DEFAULT_NAME,
+          this.hiveMetastoreUrl,
+          1,
+          new Configuration()
+      )) {
+        Table table;
+        try {
+          table = catalog.loadTable(this.tableIdentifier);
+        } catch (NoSuchTableException e) {
+          // If it doesn't exist, we just create the table
+          table = catalog.createTable(this.tableIdentifier, schema);
+        }
+
+        // In case the schema has changed
+        if (table.schema() != this.schema) {
+          table.updateSchema().unionByNameWith(this.schema).commit();
+        }
+
+        // Append the new files
+        final AppendFiles app = table.newAppend();
+        // We need to get the statistics, not easy to get them through Beam

Review comment:
       Thanks for pointing out. I'll track PR 
https://github.com/apache/iceberg/pull/1963/files. I think we have to make a 
decision here. I do like the fact that we fully re-use the writers from Beam, 
so we don't have to maintain that part. 

##########
File path: api/src/main/java/org/apache/iceberg/catalog/Namespace.java
##########
@@ -19,13 +19,14 @@
 
 package org.apache.iceberg.catalog;
 
+import java.io.Serializable;
 import java.util.Arrays;
 import org.apache.iceberg.relocated.com.google.common.base.Joiner;
 
 /**
  * A namespace in a {@link Catalog}.
  */
-public class Namespace {

Review comment:
       Unfortunately, yes. With Beam, there is no Master/Worker as with Spark 
or Task/Job manager as with Flink. Everything happens on a worker, requiring 
everything to be serialized. For example, the interaction with the Hive 
metastore is also done by the worker. This requires everything to be serialized.




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