[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402431784
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,242 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
 
 Review comment:
   This is the actual error
   ```
   java.lang.NoSuchMethodError: 
com.google.common.base.Objects.toStringHelper(Ljava/lang/Object;)Lcom/google/common/base/Objects$ToStringHelper;
   
at 
org.apache.hadoop.metrics2.lib.MetricsRegistry.toString(MetricsRegistry.java:406)
at java.lang.String.valueOf(String.java:2994)
at java.lang.StringBuilder.append(StringBuilder.java:131)
at org.apache.hadoop.ipc.metrics.RpcMetrics.(RpcMetrics.java:74)
at org.apache.hadoop.ipc.metrics.RpcMetrics.create(RpcMetrics.java:80)
at org.apache.hadoop.ipc.Server.(Server.java:2218)
at org.apache.hadoop.ipc.RPC$Server.(RPC.java:951)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server.(ProtobufRpcEngine.java:534)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine.getServer(ProtobufRpcEngine.java:509)
at org.apache.hadoop.ipc.RPC$Builder.build(RPC.java:796)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.(NameNodeRpcServer.java:351)
at 
org.apache.hadoop.hdfs.server.namenode.NameNode.createRpcServer(NameNode.java:674)
   ```
   I have the MR framework added in my own repo 
https://github.com/rdsr/incubator-iceberg/blob/mr_generic_job/mr/src/test/java/org/apache/iceberg/mr/TestMRJob.java


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402441867
 
 

 ##
 File path: build.gradle
 ##
 @@ -193,6 +193,24 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-hive')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
 
 Review comment:
   I've removed iceberg-hive altogether. Relying on HadoopCatalog to test the 
Catalog plugin


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402431784
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,242 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
 
 Review comment:
   This is the actual error
   ```
   java.lang.NoSuchMethodError: 
com.google.common.base.Objects.toStringHelper(Ljava/lang/Object;)Lcom/google/common/base/Objects$ToStringHelper;
   
at 
org.apache.hadoop.metrics2.lib.MetricsRegistry.toString(MetricsRegistry.java:406)
at java.lang.String.valueOf(String.java:2994)
at java.lang.StringBuilder.append(StringBuilder.java:131)
at org.apache.hadoop.ipc.metrics.RpcMetrics.(RpcMetrics.java:74)
at org.apache.hadoop.ipc.metrics.RpcMetrics.create(RpcMetrics.java:80)
at org.apache.hadoop.ipc.Server.(Server.java:2218)
at org.apache.hadoop.ipc.RPC$Server.(RPC.java:951)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server.(ProtobufRpcEngine.java:534)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine.getServer(ProtobufRpcEngine.java:509)
at org.apache.hadoop.ipc.RPC$Builder.build(RPC.java:796)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.(NameNodeRpcServer.java:351)
at 
org.apache.hadoop.hdfs.server.namenode.NameNode.createRpcServer(NameNode.java:674)
   ```
   I've the MR framework added in my own repo 
https://github.com/rdsr/incubator-iceberg/blob/mr_generic_job/mr/src/test/java/org/apache/iceberg/mr/TestMRJob.java


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402431784
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,242 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
 
 Review comment:
   This is the actual error
   ```
   java.lang.NoSuchMethodError: 
com.google.common.base.Objects.toStringHelper(Ljava/lang/Object;)Lcom/google/common/base/Objects$ToStringHelper;
   
at 
org.apache.hadoop.metrics2.lib.MetricsRegistry.toString(MetricsRegistry.java:406)
at java.lang.String.valueOf(String.java:2994)
at java.lang.StringBuilder.append(StringBuilder.java:131)
at org.apache.hadoop.ipc.metrics.RpcMetrics.(RpcMetrics.java:74)
at org.apache.hadoop.ipc.metrics.RpcMetrics.create(RpcMetrics.java:80)
at org.apache.hadoop.ipc.Server.(Server.java:2218)
at org.apache.hadoop.ipc.RPC$Server.(RPC.java:951)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server.(ProtobufRpcEngine.java:534)
at 
org.apache.hadoop.ipc.ProtobufRpcEngine.getServer(ProtobufRpcEngine.java:509)
at org.apache.hadoop.ipc.RPC$Builder.build(RPC.java:796)
at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.(NameNodeRpcServer.java:351)
at 
org.apache.hadoop.hdfs.server.namenode.NameNode.createRpcServer(NameNode.java:674)
   ```
   I've the MR framework tested in my own repo 
https://github.com/rdsr/incubator-iceberg/blob/mr_generic_job/mr/src/test/java/org/apache/iceberg/mr/TestMRJob.java


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402418755
 
 

 ##
 File path: 
mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java
 ##
 @@ -346,14 +348,15 @@ public void close() throws IOException {
   DataFile file = currentTask.file();
   // schema of rows returned by readers
   PartitionSpec spec = currentTask.spec();
-  Set idColumns = spec.identitySourceIds();
   Schema readSchema = expectedSchema != null ? expectedSchema : 
tableSchema;
+  Set idColumns =  Sets.intersection(spec.identitySourceIds(), 
TypeUtil.getProjectedIds(readSchema));
   boolean hasJoinedPartitionColumns = !idColumns.isEmpty();
+
   if (hasJoinedPartitionColumns) {
-readSchema = TypeUtil.selectNot(tableSchema, idColumns);
 
 Review comment:
   Had to change this logic slightly so that whatever schema the user 
projected, that is what is returned [we drop identity partition columns not 
projected by the user]


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402099264
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,567 @@
+/*
+ * 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.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Generic Mrv2 InputFormat API for Iceberg.
+ * @param  T is the in memory data model which can either be Pig tuples, 
Hive rows. Default is Iceberg records
+ */
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String LOCALITY = "iceberg.mr.locality";
+  static final String CATALOG = "iceberg.mr.catalog";
+  static final String PLATFORM_APPLIES_FILTER_RESIDUALS = 
"iceberg.mr.platform.applies.filter.residuals";
+
+  private transient List splits;
+
+  private enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+  

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402075912
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,242 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
 
 Review comment:
   I tried writing a test with DFS and Mini MR cluster. I'm facing class-path 
issues particularly with Guava. Any suggestions on how to go about it? I was 
thinking maybe we can have an integration testing module per  framework. For 
example for MR we'd have `mr-runtime` module which would be shaded. Now to test 
MR we could have a `mr-runtime-test` module which can setup DFS and MR 
clusters. Since mr runtime would be shaded completely there wouldn't be any 
classpath issues
   I've added more unit test though.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402076499
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
 
 Review comment:
   Fixed.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-04-02 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r402075912
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,242 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
 
 Review comment:
   I tried writing a test with DFS and MR cluster. I'm facing classpath issues 
particularly with Guava. Any suggestions on how to go about it? I was think 
maybe we can have an integration testing module per  framework. For example for 
MR we'd have `mr-runtime` module which would be shaded. Now to test MR we could 
have a `mr-runtime-test` module which can setup DFS and MR clusters. Since mr 
runtime would be shaded completely we can write these integration tests.
   
   I've added more unit test though.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397426058
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,525 @@
+/*
+ * 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.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Generic Mrv2 InputFormat API for Iceberg.
+ * @param  T is the in memory data model which can either be Pig tuples, 
Hive rows. Default is Iceberg records
+ */
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String LOCALITY = "iceberg.mr.locality";
+  static final String CATALOG = "iceberg.mr.catalog";
+
+  private transient List splits;
+
+  private enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397425345
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397425345
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397419634
 
 

 ##
 File path: build.gradle
 ##
 @@ -219,6 +219,58 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+compile project(':iceberg-data')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(':iceberg-hive')
+testCompile project(path: ':iceberg-data', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-api', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
+
+// By default, hive-exec is a fat/uber jar and it exports a guava library
+// that's really old. We use the core classifier to be able to override 
our guava
+// version. Luckily, hive-exec seems to work okay so far with this version 
of guava
+// See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for 
more context.
+testCompile("org.apache.hive:hive-exec::core") {
 
 Review comment:
   I've removed the iceberg-hive and related deps from iceberg-mr module and 
the plugin system is being tested using `HadoopCatalog`.  I can still add the 
test for HiveCatalog in the iceberg-hive module. What do you think?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397419634
 
 

 ##
 File path: build.gradle
 ##
 @@ -219,6 +219,58 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+compile project(':iceberg-data')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(':iceberg-hive')
+testCompile project(path: ':iceberg-data', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-api', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
+
+// By default, hive-exec is a fat/uber jar and it exports a guava library
+// that's really old. We use the core classifier to be able to override 
our guava
+// version. Luckily, hive-exec seems to work okay so far with this version 
of guava
+// See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for 
more context.
+testCompile("org.apache.hive:hive-exec::core") {
 
 Review comment:
   I've remove the iceberg-hive and related deps from iceberg-mr module and the 
plugin system is being tested using `HadoopCatalog`.  I can still add the test 
for HiveCatalog in the iceberg-hive module. What do you think?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397419046
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,525 @@
+/*
+ * 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.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Generic Mrv2 InputFormat API for Iceberg.
+ * @param  T is the in memory data model which can either be Pig tuples, 
Hive rows. Default is Iceberg records
+ */
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String LOCALITY = "iceberg.mr.locality";
+  static final String CATALOG = "iceberg.mr.catalog";
+
+  private transient List splits;
+
+  private enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397416725
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-24 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r397386397
 
 

 ##
 File path: build.gradle
 ##
 @@ -219,6 +219,58 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+compile project(':iceberg-data')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(':iceberg-hive')
+testCompile project(path: ':iceberg-data', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-api', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
+
+// By default, hive-exec is a fat/uber jar and it exports a guava library
+// that's really old. We use the core classifier to be able to override 
our guava
+// version. Luckily, hive-exec seems to work okay so far with this version 
of guava
+// See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for 
more context.
+testCompile("org.apache.hive:hive-exec::core") {
 
 Review comment:
   We'd need to depend on iceberg-hive in test scope so that we can use 
HiveCatalog


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-22 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396174856
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,525 @@
+/*
+ * 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.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Generic Mrv2 InputFormat API for Iceberg.
+ * @param  T is the in memory data model which can either be Pig tuples, 
Hive rows. Default is Iceberg records
+ */
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String LOCALITY = "iceberg.mr.locality";
+  static final String CATALOG = "iceberg.mr.catalog";
+
+  private transient List splits;
+
+  private enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-22 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396174856
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,525 @@
+/*
+ * 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.mr;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynConstructors;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hadoop.Util;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Generic Mrv2 InputFormat API for Iceberg.
+ * @param  T is the in memory data model which can either be Pig tuples, 
Hive rows. Default is Iceberg records
+ */
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String LOCALITY = "iceberg.mr.locality";
+  static final String CATALOG = "iceberg.mr.catalog";
+
+  private transient List splits;
+
+  private enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-22 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396172831
 
 

 ##
 File path: build.gradle
 ##
 @@ -219,6 +219,58 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+compile project(':iceberg-data')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(':iceberg-hive')
+testCompile project(path: ':iceberg-data', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-api', configuration: 'testArtifacts')
+testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
+
+// By default, hive-exec is a fat/uber jar and it exports a guava library
+// that's really old. We use the core classifier to be able to override 
our guava
+// version. Luckily, hive-exec seems to work okay so far with this version 
of guava
+// See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for 
more context.
+testCompile("org.apache.hive:hive-exec::core") {
 
 Review comment:
   I'l also worried that iceberg-hive in test scope can create a circular dep. 
Maybe we can just test with `HadoopCatalog` ?


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396016799
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,260 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.hive.HiveClientPool;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
+  private static final Schema SCHEMA = new Schema(
+  required(1, "data", Types.StringType.get()),
+  required(3, "id", Types.LongType.get()),
+  required(2, "date", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+ .identity("date")
+ .bucket("id", 1)
+ .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  private HadoopTables tables;
+  private Configuration conf;
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+return new Object[][]{
+new Object[]{"parquet"},
+new Object[]{"avro"}
 
 Review comment:
   Will add orc once ORC patches are committed which read/writer Iceberg 
generics


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396016799
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,260 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.FluentIterable;
+import com.google.common.collect.ImmutableMap;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataWriter;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.hive.HiveClientPool;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+@RunWith(Parameterized.class)
+public class TestIcebergInputFormat {
+  private static final Schema SCHEMA = new Schema(
+  required(1, "data", Types.StringType.get()),
+  required(3, "id", Types.LongType.get()),
+  required(2, "date", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+ .identity("date")
+ .bucket("id", 1)
+ .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  private HadoopTables tables;
+  private Configuration conf;
+
+  @Parameterized.Parameters
+  public static Object[][] parameters() {
+return new Object[][]{
+new Object[]{"parquet"},
+new Object[]{"avro"}
 
 Review comment:
   Will add orc once ORC patches are committed which read/writer Icebreg 
generics


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396016593
 
 

 ##
 File path: build.gradle
 ##
 @@ -193,6 +193,24 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-hive')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
 
 Review comment:
   Maybe iceberg-hive as a test dep can also cause recursive deps. If that's 
the case we can simply test with `HadoopCatalog` instead and remove 
iceberg-hive altogether


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396012516
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,133 @@
+/*
+ * 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.mr;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.RandomAvroData;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestIcebergInputFormat {
+  private static final Configuration CONF = new Configuration();
+  private static final HadoopTables TABLES = new HadoopTables(CONF);
+
+  private File tableLocation;
+
+  private static final Schema SCHEMA = new Schema(
+  required(1, "id", Types.LongType.get()),
+  optional(2, "data", Types.StringType.get()),
+  required(3, "date", Types.StringType.get()));
+
+  private static final PartitionSpec PARTITION_BY_DATE = PartitionSpec
+  .builderFor(SCHEMA)
+  .identity("date")
+  .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  IcebergInputFormat icebergInputFormat;
+
+  @Test
+  public void test() throws IOException, InterruptedException {
+tableLocation = new File(temp.newFolder(), "table");
+Table table = TABLES.create(SCHEMA, PARTITION_BY_DATE, 
tableLocation.toString());
+List records = RandomAvroData.generate(SCHEMA, 5, 0L);
+File file = temp.newFile();
+Assert.assertTrue(file.delete());
+try (FileAppender appender = 
Avro.write(Files.localOutput(file))
+ .schema(SCHEMA)
+ .named("avro")
+ .build()) {
+  appender.addAll(records);
+}
+
+DataFile dataFile = DataFiles.builder(PARTITION_BY_DATE)
+ .withPartition(partitionData("2020-03-15"))
+ .withRecordCount(records.size())
+ .withFileSizeInBytes(file.length())
+ .withPath(file.toString())
+ .withFormat("avro")
+ .build();
+
+table.newAppend().appendFile(dataFile).commit();
+
+Job job = Job.getInstance(new Configuration());
+IcebergInputFormat
+.configure(job)
+.readFrom(tableLocation.getAbsolutePath());
+
+TaskAttemptContext context = new TaskAttemptContextImpl(new 
JobConf(job.getConfiguration()), new TaskAttemptID());
+icebergInputFormat = new IcebergInputFormat<>();
+List splits = icebergInputFormat.getSplits(context);
+final RecordReader recordReader =
+icebergInputFormat.createRecordReader(splits.get(0), context);
+recordReader.initialize(splits.get(0), context);
+while (recordReader.nextKeyValue()) {
+  System.out.println(recordReader.getCurrentValue());
 
 Review comment:
   Fixed.


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396012478
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396012124
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,345 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
 
 Review comment:
   Added.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396012059
 
 

 ##
 File path: build.gradle
 ##
 @@ -193,6 +193,24 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-hive')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
 
 Review comment:
   I fixed this. For now, I've adding iceberg-hive as a test dep


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-21 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r396011822
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
 
 Review comment:
   @massdosage . I'll make this change once all other comments are addressed. I 
think making this change before can trip up github and it may consider 
mr.InputFormat and mr.mapreduce.InputFormat as two different files.


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-19 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r395389858
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-19 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r395389858
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-19 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r395374741
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-18 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r394484945
 
 

 ##
 File path: mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormat.java
 ##
 @@ -0,0 +1,133 @@
+/*
+ * 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.mr;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import org.apache.avro.generic.GenericData;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.avro.RandomAvroData;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class TestIcebergInputFormat {
+  private static final Configuration CONF = new Configuration();
+  private static final HadoopTables TABLES = new HadoopTables(CONF);
+
+  private File tableLocation;
+
+  private static final Schema SCHEMA = new Schema(
+  required(1, "id", Types.LongType.get()),
+  optional(2, "data", Types.StringType.get()),
+  required(3, "date", Types.StringType.get()));
+
+  private static final PartitionSpec PARTITION_BY_DATE = PartitionSpec
+  .builderFor(SCHEMA)
+  .identity("date")
+  .build();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  IcebergInputFormat icebergInputFormat;
+
+  @Test
+  public void test() throws IOException, InterruptedException {
+tableLocation = new File(temp.newFolder(), "table");
+Table table = TABLES.create(SCHEMA, PARTITION_BY_DATE, 
tableLocation.toString());
+List records = RandomAvroData.generate(SCHEMA, 5, 0L);
+File file = temp.newFile();
+Assert.assertTrue(file.delete());
+try (FileAppender appender = 
Avro.write(Files.localOutput(file))
+ .schema(SCHEMA)
+ .named("avro")
+ .build()) {
+  appender.addAll(records);
+}
+
+DataFile dataFile = DataFiles.builder(PARTITION_BY_DATE)
+ .withPartition(partitionData("2020-03-15"))
+ .withRecordCount(records.size())
+ .withFileSizeInBytes(file.length())
+ .withPath(file.toString())
+ .withFormat("avro")
+ .build();
+
+table.newAppend().appendFile(dataFile).commit();
+
+Job job = Job.getInstance(new Configuration());
+IcebergInputFormat
+.configure(job)
+.readFrom(tableLocation.getAbsolutePath());
+
+TaskAttemptContext context = new TaskAttemptContextImpl(new 
JobConf(job.getConfiguration()), new TaskAttemptID());
+icebergInputFormat = new IcebergInputFormat<>();
+List splits = icebergInputFormat.getSplits(context);
+final RecordReader recordReader =
+icebergInputFormat.createRecordReader(splits.get(0), context);
+recordReader.initialize(splits.get(0), context);
+while (recordReader.nextKeyValue()) {
+  System.out.println(recordReader.getCurrentValue());
 
 Review comment:
   Tests are not done well. I plan to write better tests before 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-18 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r394483967
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-18 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r394477007
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-18 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r394477007
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,494 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String AS_OF_TIMESTAMP = "iceberg.mr.as.of.time";
+  static final String CASE_SENSITIVE = "iceberg.mr.case.sensitive";
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String IN_MEMORY_DATA_MODEL = "iceberg.mr.in.memory.data.model";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String REUSE_CONTAINERS = "iceberg.mr.case.sensitive";
+  static final String SNAPSHOT_ID = "iceberg.mr.snapshot.id";
+  static final String SPLIT_SIZE = "iceberg.mr.split.size";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  private static final String LOCALITY = "iceberg.mr.locality";
+
+  private transient List splits;
+
+  public enum InMemoryDataModel {
+PIG,
+HIVE,
+DEFAULT // Default data model is of Iceberg Generics
+  }
+
+  /**
+   * Configures the {@code Job} to use the {@code IcebergInputFormat} and
+   * returns a helper to add further configuration.
+   *
+   * @param job the {@code Job} to configure
+   */
+  public static ConfigBuilder configure(Job job) {
+job.setInputFormatClass(IcebergInputFormat.class);
+return new ConfigBuilder(job.getConfiguration());
+  }
+
+  public static class ConfigBuilder {
+private final Configuration conf;
+
+public ConfigBuilder(Configuration conf) {
+  this.conf = conf;
+}
+
+public ConfigBuilder readFrom(String path) {
+  conf.set(TABLE_PATH, path);
+  Table 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-18 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r394339254
 
 

 ##
 File path: build.gradle
 ##
 @@ -193,6 +193,24 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-hive')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
 
 Review comment:
   We'd need the `org.apache.iceberg.hive.HiveCatalogs#loadCatalog` which is 
available in iceberg-hive


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393228347
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,345 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String READ_SUPPORT = "iceberg.mr.read.support";
+
+  private transient Table table;
+  private transient List splits;
+
+  public IcebergInputFormat() {
+  }
+
+  @Override
+  public List getSplits(JobContext context) {
+if (splits != null) {
+  LOG.info("Returning cached splits: {}", splits.size());
+  return splits;
+}
+
+Configuration conf = context.getConfiguration();
+table = getTable(conf);
+TableScan scan = table.newScan();
+//TODO add caseSensitive, snapshot id etc..
+
+Expression filterExpression = 
SerializationUtil.deserializeFromBase64(conf.get(FILTER_EXPRESSION));
+if (filterExpression != null) {
+  scan = scan.filter(filterExpression);
+}
+
+final String schemaStr = conf.get(READ_SCHEMA);
+if (schemaStr != null) {
+  // Not sure if this is having any effect?
+  scan.project(SchemaParser.fromJson(schemaStr));
+}
+
+splits = Lists.newArrayList();
+try (CloseableIterable tasksIterable = scan.planTasks()) 
{
+  tasksIterable.forEach(task -> splits.add(new IcebergSplit(task)));
+} catch (IOException e) {
+  throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
+}
+
+return splits;
+  }
+
+  @Override
+  public RecordReader createRecordReader(InputSplit split, 
TaskAttemptContext context) {
+return new IcebergRecordReader();
+  }
+
+  public static ConfBuilder updateConf(
 
 Review comment:
   This looks great. Will use


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:
us...@infra.apache.org

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393224767
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,345 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String READ_SUPPORT = "iceberg.mr.read.support";
+
+  private transient Table table;
+  private transient List splits;
+
+  public IcebergInputFormat() {
+  }
+
+  @Override
+  public List getSplits(JobContext context) {
+if (splits != null) {
+  LOG.info("Returning cached splits: {}", splits.size());
+  return splits;
+}
+
+Configuration conf = context.getConfiguration();
+table = getTable(conf);
+TableScan scan = table.newScan();
+//TODO add caseSensitive, snapshot id etc..
+
+Expression filterExpression = 
SerializationUtil.deserializeFromBase64(conf.get(FILTER_EXPRESSION));
+if (filterExpression != null) {
+  scan = scan.filter(filterExpression);
+}
+
+final String schemaStr = conf.get(READ_SCHEMA);
+if (schemaStr != null) {
+  // Not sure if this is having any effect?
+  scan.project(SchemaParser.fromJson(schemaStr));
+}
+
+splits = Lists.newArrayList();
+try (CloseableIterable tasksIterable = scan.planTasks()) 
{
+  tasksIterable.forEach(task -> splits.add(new IcebergSplit(task)));
+} catch (IOException e) {
+  throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
+}
+
+return splits;
+  }
+
+  @Override
+  public RecordReader createRecordReader(InputSplit split, 
TaskAttemptContext context) {
+return new IcebergRecordReader();
+  }
+
+  public static ConfBuilder updateConf(
+  Configuration conf, String path, Class> 
readSupportClass) {
+return new ConfBuilder(conf, path, readSupportClass);
+  }
+
+  public static class ConfBuilder {
+private final Configuration conf;
+
+public ConfBuilder(Configuration conf, String path, Class> readSupportClass) {
 
 Review comment:
   in-memory data model +1.
   For table path, which 

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393223221
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,345 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String READ_SUPPORT = "iceberg.mr.read.support";
+
+  private transient Table table;
+  private transient List splits;
+
+  public IcebergInputFormat() {
+  }
+
+  @Override
+  public List getSplits(JobContext context) {
+if (splits != null) {
+  LOG.info("Returning cached splits: {}", splits.size());
+  return splits;
+}
+
+Configuration conf = context.getConfiguration();
+table = getTable(conf);
+TableScan scan = table.newScan();
+//TODO add caseSensitive, snapshot id etc..
+
+Expression filterExpression = 
SerializationUtil.deserializeFromBase64(conf.get(FILTER_EXPRESSION));
+if (filterExpression != null) {
+  scan = scan.filter(filterExpression);
+}
+
+final String schemaStr = conf.get(READ_SCHEMA);
+if (schemaStr != null) {
+  // Not sure if this is having any effect?
+  scan.project(SchemaParser.fromJson(schemaStr));
+}
+
+splits = Lists.newArrayList();
+try (CloseableIterable tasksIterable = scan.planTasks()) 
{
+  tasksIterable.forEach(task -> splits.add(new IcebergSplit(task)));
+} catch (IOException e) {
+  throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
+}
+
+return splits;
+  }
+
+  @Override
+  public RecordReader createRecordReader(InputSplit split, 
TaskAttemptContext context) {
+return new IcebergRecordReader();
+  }
+
+  public static ConfBuilder updateConf(
+  Configuration conf, String path, Class> 
readSupportClass) {
+return new ConfBuilder(conf, path, readSupportClass);
+  }
+
+  public static class ConfBuilder {
+private final Configuration conf;
+
+public ConfBuilder(Configuration conf, String path, Class> readSupportClass) {
+  this.conf = conf;
+  conf.set(TABLE_PATH, path);
+  

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393222877
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/ReadSupport.java
 ##
 @@ -0,0 +1,50 @@
+package org.apache.iceberg.mr;
+
+import java.util.function.BiFunction;
+import org.apache.avro.io.DatumReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.orc.OrcValueReader;
+import org.apache.iceberg.parquet.ParquetValueReader;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.orc.TypeDescription;
+import org.apache.parquet.schema.MessageType;
+
+import java.util.function.Function;
+
+
+/**
+ * ReadSupport for MR InputFormat, providing value readers
+ * for different data formats and appending identity partition columns
+ * to the input row
+ * @param 
+ */
+public interface ReadSupport {
 
 Review comment:
   I think that makes sense. We either allow Pig's `Tuples`, Hive's 
`ObjectInspectors` or default Iceberg `GenericRecords`


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:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393003639
 
 

 ##
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##
 @@ -0,0 +1,345 @@
+/*
+ * 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.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat extends InputFormat {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String READ_SUPPORT = "iceberg.mr.read.support";
+
+  private transient Table table;
+  private transient List splits;
+
+  public IcebergInputFormat() {
+  }
+
+  @Override
+  public List getSplits(JobContext context) {
+if (splits != null) {
+  LOG.info("Returning cached splits: {}", splits.size());
+  return splits;
+}
+
+Configuration conf = context.getConfiguration();
+table = getTable(conf);
+TableScan scan = table.newScan();
+//TODO add caseSensitive, snapshot id etc..
+
+Expression filterExpression = 
SerializationUtil.deserializeFromBase64(conf.get(FILTER_EXPRESSION));
+if (filterExpression != null) {
+  scan = scan.filter(filterExpression);
+}
+
+final String schemaStr = conf.get(READ_SCHEMA);
+if (schemaStr != null) {
+  // Not sure if this is having any effect?
+  scan.project(SchemaParser.fromJson(schemaStr));
+}
+
+splits = Lists.newArrayList();
+try (CloseableIterable tasksIterable = scan.planTasks()) 
{
+  tasksIterable.forEach(task -> splits.add(new IcebergSplit(task)));
+} catch (IOException e) {
+  throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
+}
+
+return splits;
+  }
+
+  @Override
+  public RecordReader createRecordReader(InputSplit split, 
TaskAttemptContext context) {
+return new IcebergRecordReader();
+  }
+
+  public static ConfBuilder updateConf(
+  Configuration conf, String path, Class> 
readSupportClass) {
+return new ConfBuilder(conf, path, readSupportClass);
+  }
+
+  public static class ConfBuilder {
+private final Configuration conf;
+
+public ConfBuilder(Configuration conf, String path, Class> readSupportClass) {
+  this.conf = conf;
+  conf.set(TABLE_PATH, path);
+  

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #843: [WIP] InputFormat support for Iceberg

2020-03-16 Thread GitBox
rdsr commented on a change in pull request #843: [WIP] InputFormat support for 
Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/843#discussion_r393000784
 
 

 ##
 File path: build.gradle
 ##
 @@ -193,6 +193,24 @@ project(':iceberg-hive') {
   }
 }
 
+project(':iceberg-mr') {
+  dependencies {
+compile project(':iceberg-api')
+compile project(':iceberg-core')
+compile project(':iceberg-hive')
+compile project(':iceberg-orc')
+compile project(':iceberg-parquet')
+
+compileOnly("org.apache.hadoop:hadoop-client") {
+  exclude group: 'org.apache.avro', module: 'avro'
+}
+
+testCompile project(path: ':iceberg-hive', configuration: 'testArtifacts')
 
 Review comment:
   This is so that I can rely on `HiveCatalog`. Yes you are right though, we'd 
have to figure out how to arrange out modules when we have a Hive storage 
handler


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:
us...@infra.apache.org


With regards,
Apache Git Services

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