You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/04/16 15:14:20 UTC

[GitHub] [incubator-iceberg] massdosage opened a new pull request #933: Mapred input format

massdosage opened a new pull request #933: Mapred input format
URL: https://github.com/apache/incubator-iceberg/pull/933
 
 
   f

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440707663



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Ah yeah, good shout on the comment and using the correct granularity, I'll make those changes!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] guilload edited a comment on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
guilload edited a comment on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-642861625


   Hi @massdosage, following @rdblue's suggestion, I'd like to open a PR against this one to bring the improvements I've made on #1104 but before I do that, do you think you could rebase this branch onto master and clean up / squash some commits to make things easier for me. This PR has currently a lot of commits and is hard to navigate.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439086089



##########
File path: mr/src/main/java/org/apache/iceberg/mr/IcebergRecordReader.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.orc.GenericOrcReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.expressions.Evaluator;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+public class IcebergRecordReader<T> {

Review comment:
       Is it possible to reuse the IcebergRecordReader already implemented in https://github.com/apache/iceberg/blob/cad124967ceb740178a3f65b75f19017494b430d/mr/src/main/java/org/apache/iceberg/mr/mapreduce/IcebergInputFormat.java#L280. Seems much more extensive to me, e.g. handling identity partitioned data and multiple data models e.g. Pig.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotsTable;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.apache.iceberg.mr.mapred.iterables.SnapshotIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  private Table table;
+  private long currentSnapshotId;
+  private String virtualSnapshotIdColumnName;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = TableResolver.resolveTableFromJob(conf);
+    URI location = TableResolver.pathAsURI(conf.get(InputFormatConfig.TABLE_LOCATION));
+    List<CombinedScanTask> tasks = planTasks(conf);
+    return createSplits(tasks, location.toString());
+  }
+
+  private List<CombinedScanTask> planTasks(JobConf conf) {
+    // Set defaults for virtual column
+    Snapshot currentSnapshot = table.currentSnapshot();
+    if (currentSnapshot != null) {
+      currentSnapshotId = currentSnapshot.snapshotId();
+    }
+    virtualSnapshotIdColumnName = SystemTableUtil.getVirtualColumnName(conf);
+
+    String[] readColumns = ColumnProjectionUtils.getReadColumnNames(conf);
+    List<CombinedScanTask> tasks;
+    if (conf.get(TableScanDesc.FILTER_EXPR_CONF_STR) == null) {
+      tasks = Lists.newArrayList(table
+              .newScan()
+              .select(readColumns)
+              .planTasks());
+    } else {
+      ExprNodeGenericFuncDesc exprNodeDesc = SerializationUtilities
+              .deserializeObject(conf.get(TableScanDesc.FILTER_EXPR_CONF_STR), ExprNodeGenericFuncDesc.class);
+      SearchArgument sarg = ConvertAstToSearchArg.create(conf, exprNodeDesc);
+      Expression filter = IcebergFilterFactory.generateFilterExpression(sarg);
+
+      long snapshotIdToScan = extractSnapshotID(conf, exprNodeDesc);

Review comment:
       I guess this may cause issues if the table itself contains a column called `snapshot__id`. @rdblue Do you think we should reserve a few column names (or a prefix) in the spec for these virtual columns? I guess such virtual columns are generally useful for supporting time travel/versioning/incremental scans in purely SQL engines.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {

Review comment:
       Can also be true for `OR`?
   
   Also, maybe simplify this to
   ```java
   Expression result = Expression.alwaysTrue();
   for (ExpressionTree child: childNodes) {
      result = and(result, translate(child, leaves))
   }
   ```

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    if (column.equals("snapshot__id")) {
+      return Expressions.alwaysTrue();
+    }
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leaf.getLiteral());

Review comment:
       We will need to convert literal values from the Hive data types to Iceberg data types.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {

Review comment:
       I guess this should be unnecessary if the logic for OR/AND operators is simplified to use a for loop as in my suggestion above.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/iterables/SnapshotIterable.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.mapred.iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+
+/**
+ * Creates an Iterable of Records with all snapshot metadata that can be used with the RecordReader.
+ */
+public class SnapshotIterable implements CloseableIterable {

Review comment:
       Is this really needed? Can we reuse https://github.com/apache/iceberg/blob/cad124967ceb740178a3f65b75f19017494b430d/core/src/main/java/org/apache/iceberg/SnapshotsTable.java#L60 to get the iterable instead?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;

Review comment:
       We return `null` here but `Expression.and()`, `Expressions.not()` and Expressions.or()` have null checks in them, so seems like this will fail regardless? Is it better to throw a more readable exception here instead.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {

Review comment:
       A lot of the code is duplicated between this method and `translate()`. Should we just call `translate(sarg.getExpression(), sarg.getLeaves())` here?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    if (column.equals("snapshot__id")) {

Review comment:
       Reference the `snapshot__id` constant defined in SystemTableUtil

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/TableResolver.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.mapred;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+final class TableResolver {

Review comment:
       I don't see the need to for having two table resolver classes. Can we combine them? The only difference I see is that this class uses `org.apache.hadoop.mapred.JobConf` instead of `org.apache.hadoop.mapreduce.JobConf`. It seems like we don't use any `JobConf` methods, and if we have to, we can create 2 methods instead of 2 classes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440058712



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/iterables/SnapshotIterable.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.mapred.iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+
+/**
+ * Creates an Iterable of Records with all snapshot metadata that can be used with the RecordReader.
+ */
+public class SnapshotIterable implements CloseableIterable {

Review comment:
       A great call out, that looks like exactly the sort of thing we should be doing here! I'll remove most of the system tables stuff from our PR's and we'll do a separate PR that should be easier to review :))




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440702265



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Ugh! Nice find. `Timestamp.getTime()` is probably still incorrect though as Iceberg expects the number of days since epoch as a literal for the Date type. `Timestamp.getTime()` will give you the number of mills since epoch.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439585809



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -131,7 +90,7 @@ private static Expression translate(ExpressionTree tree, List<PredicateLeaf> lea
    */
   private static Expression translateLeaf(PredicateLeaf leaf) {
     String column = leaf.getColumnName();
-    if (column.equals("snapshot__id")) {
+    if (column.equals(SystemTableUtil.DEFAULT_SNAPSHOT_ID_COLUMN_NAME)) {

Review comment:
       Actually, this may not be enough since the column name is configurable?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -96,29 +54,30 @@ public static Expression generateFilterExpression(SearchArgument sarg) {
    * @param leaves List of all leaf nodes within the tree.
    * @return Expression that is translated from the Hive SearchArgument.
    */
-  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves,

Review comment:
       Nit: childNodes doesn't need to be parameter, can be a local variable instead.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -96,29 +54,30 @@ public static Expression generateFilterExpression(SearchArgument sarg) {
    * @param leaves List of all leaf nodes within the tree.
    * @return Expression that is translated from the Hive SearchArgument.
    */
-  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves,
+                                      List<ExpressionTree> childNodes) {
     switch (tree.getOperator()) {
       case OR:
-        return or(translate(tree.getChildren().get(0), leaves),
-                translate(tree.getChildren().get(1), leaves));
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves, childNodes));

Review comment:
       I think passing `childNodes` here is incorrect. It should be `child.getChildren()` else we just keep on passing the root child nodes over and over. However, I think we should just make `childNodes` a local variable instead so that we don't make this mistake. It would also be good to add some tests for nested expressions.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439453405



##########
File path: mr/src/main/java/org/apache/iceberg/mr/IcebergRecordReader.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.orc.GenericOrcReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.expressions.Evaluator;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+public class IcebergRecordReader<T> {

Review comment:
       This intention here is for common record reader code across both the `mapred` and `mapreduce` sub-packages to be here and then the specific implementations just deal with any particulars related to the different APIs. Right now the InputFormat in the `mapred` package is only used by Hive. I'd be happy to move more common code out as and when we find it, this is just a start at how it could be done. I'd prefer to do that in later PRs and get a working. albeit basic, end to end read path for Hive merged first.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] pvary commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
pvary commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r458719148



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       It is not released yet, but later we have to consider the TimeZones as well. See: https://issues.apache.org/jira/browse/HIVE-20007




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439536799



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    if (column.equals("snapshot__id")) {
+      return Expressions.alwaysTrue();
+    }
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leaf.getLiteral());

Review comment:
       I'm not entirely sure what this means - is this for things like Dates and Timestamps to be of the right type? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] guilload commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
guilload commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-642861625


   Hi @massdosage, following @rdblue's suggestion, I'd like to open a PR against this one to bring the improvements I've made on #1104 but before I that, do you think you could rebase this branch onto master and clean up / squash some commits to make things easier for me. This PR has currently a lot of commits and is hard to navigate.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-643320072


   > Hi @massdosage, following @rdblue's suggestion, I'd like to open a PR against this one to bring the improvements I've made on #1104 but before I do that, do you think you could rebase this branch onto master and clean up / squash some commits to make things easier for me. This PR has currently a lot of commits and is hard to navigate.
   
   I was assuming it will get squashed when merged into master so the (too many) commits would get removed at that stage. Is there a specific problem on your side? If we rebase it that will break everyone's existing checkouts.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r435887751



##########
File path: mr/dependencies.lock
##########
@@ -1,4 +1,365 @@
 {
+    "allProcessors": {

Review comment:
       OK, I'll just look away because it does seem to be working ;)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440685238



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       IIRC Hive's Date type predicate literal is of type java.sql.Date. So this looks incorrect? (Unsure)

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Timestamp.getTime() gives back milliseconds since epoch, but Iceberg expects microsecond granularity. You might want to factor in Timestamp.getNanos() to get microsecond granularity. 

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case BOOLEAN:
+        return leaf.getLiteral();
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> hiveLiteralListToIcebergType(List<Object> hiveLiteralTypes) {

Review comment:
       We should just reuse `leafToIcebergType` here, seems like theres equivalent logic in two places.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439539602



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,306 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
+import org.apache.hadoop.hive.ql.plan.TableScanDesc;
+import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.SnapshotsTable;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.mr.InputFormatConfig;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.apache.iceberg.mr.mapred.iterables.SnapshotIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  private Table table;
+  private long currentSnapshotId;
+  private String virtualSnapshotIdColumnName;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = TableResolver.resolveTableFromJob(conf);
+    URI location = TableResolver.pathAsURI(conf.get(InputFormatConfig.TABLE_LOCATION));
+    List<CombinedScanTask> tasks = planTasks(conf);
+    return createSplits(tasks, location.toString());
+  }
+
+  private List<CombinedScanTask> planTasks(JobConf conf) {
+    // Set defaults for virtual column
+    Snapshot currentSnapshot = table.currentSnapshot();
+    if (currentSnapshot != null) {
+      currentSnapshotId = currentSnapshot.snapshotId();
+    }
+    virtualSnapshotIdColumnName = SystemTableUtil.getVirtualColumnName(conf);
+
+    String[] readColumns = ColumnProjectionUtils.getReadColumnNames(conf);
+    List<CombinedScanTask> tasks;
+    if (conf.get(TableScanDesc.FILTER_EXPR_CONF_STR) == null) {
+      tasks = Lists.newArrayList(table
+              .newScan()
+              .select(readColumns)
+              .planTasks());
+    } else {
+      ExprNodeGenericFuncDesc exprNodeDesc = SerializationUtilities
+              .deserializeObject(conf.get(TableScanDesc.FILTER_EXPR_CONF_STR), ExprNodeGenericFuncDesc.class);
+      SearchArgument sarg = ConvertAstToSearchArg.create(conf, exprNodeDesc);
+      Expression filter = IcebergFilterFactory.generateFilterExpression(sarg);
+
+      long snapshotIdToScan = extractSnapshotID(conf, exprNodeDesc);

Review comment:
       Yeah, this was an edge case that cropped up when we were testing. We got around it by making it configurable but with a default of `snapshot__id`. So by default we're adding this extra column to a table schema, but if a user knows they already have a column with this same name they can set the virtual column to a different name:
   ```
   TBLPROPERTIES ('iceberg.hive.snapshot.virtual.column.name' = 'new_column_name')
   ```
   But reserving column names would be a nice addition so this check doesn't need to happen
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439606167



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    if (column.equals("snapshot__id")) {
+      return Expressions.alwaysTrue();
+    }
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leaf.getLiteral());

Review comment:
       Yup! Heres what Hive will give you https://github.com/apache/hive/blob/cb213d88304034393d68cc31a95be24f5aac62b6/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java#L49-L56 but Iceberg expects https://github.com/linkedin/iceberg/blob/6e5601bbea2c032f11d90114e2c3e64dc6c0e5c3/api/src/main/java/org/apache/iceberg/types/Type.java#L29-L45. So you will need add conversions for the parity. Would also suggest adding one more test case which covers all datatypes that Hive can give us.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440703210



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Also, might be good to mention this issue as a comment as it looks very non-intuitive.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440697208



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       An integration test using HiveRunner will probably be very helpful in finding these conversion issues. I would suggest looking at https://github.com/apache/iceberg/blob/90d0a0e59fc146534b115f56dfe8d6d63156a198/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java#L78 which has helped me debug issues related to filter conversions in the past, especially the test dealing with Date and Timestamp types https://github.com/apache/iceberg/blob/90d0a0e59fc146534b115f56dfe8d6d63156a198/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java#L458 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r458941801



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       We usually leave time zone handling to the engine. Engines should pass concrete values to Iceberg in expressions.
   
   For example, a query might have a clause `WHERE ts > TIMESTAMP '...'`. It is the query engine's responsibility to determine what concrete value that timestamp actually represents. If `ts` is a timestamp without time zone, for example, the engine is responsible for converting the timestamp with time zone for comparison.
   
   Iceberg should receive an unambiguous value to use in the comparison. Preferably, the value is in microseconds from epoch, but we can convert from alternative representations like millis + nanos.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Iceberg leaves time zone handling to the engine. Engines should pass concrete values to Iceberg in expressions.
   
   For example, a query might have a clause `WHERE ts > TIMESTAMP '...'`. It is the query engine's responsibility to determine what concrete value that timestamp actually represents. If `ts` is a timestamp without time zone, for example, the engine is responsible for converting the timestamp with time zone for comparison.
   
   Iceberg should receive an unambiguous value to use in the comparison. Preferably, the value is in microseconds from epoch, but we can convert from alternative representations like millis + nanos.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-639452092


   > Looks like the current problem is that missing `org.pentaho` Jar again. Maybe we should move that exclusion into the `configurations.all` block so it's global.
   
   OK, I don't have that issue (probably because I have the jar cached locally) but I know what you're talking about as I've seen it in many other projects. I'll try remove it locally and see if I can reproduce it and then add an exclusion like you suggest.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-662947955


   We adapted our approach and worked with @guilload on #1192 which has now been merged so I'm going to close this. We'll raise follow-on PRs which contain any missing functionality that was here as well as things we removed from this PR to simplify it (pushdowns, time travel queries etc.)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage closed pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage closed pull request #933:
URL: https://github.com/apache/iceberg/pull/933


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439520651



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {

Review comment:
       Yeah a great suggestion, I'll make those changes :D




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r435232380



##########
File path: mr/dependencies.lock
##########
@@ -1,4 +1,365 @@
 {
+    "allProcessors": {

Review comment:
       This `allProcessors` block doesn't look right. @rdblue I ran ` ./gradlew :iceberg-mr:generateLock` and then ` ./gradlew :iceberg-mr:saveLock` to generate this file. Did I miss something?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439103257



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/TableResolver.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.mapred;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+final class TableResolver {

Review comment:
       I don't see the need for having two table resolver classes. Can we combine them? The only difference I see is that this class uses `org.apache.hadoop.mapred.JobConf` instead of `org.apache.hadoop.mapreduce.JobConf`. It seems like we don't use any `JobConf` methods, and if we have to, we can create 2 methods instead of 2 classes.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/TableResolver.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.mapred;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+final class TableResolver {

Review comment:
       I don't see the need for having two TableResolver classes. Can we combine them? The only difference I see is that this class uses `org.apache.hadoop.mapred.JobConf` instead of `org.apache.hadoop.mapreduce.JobConf`. It seems like we don't use any `JobConf` methods, and if we have to, we can create 2 methods instead of 2 classes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439636404



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/iterables/SnapshotIterable.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.mapred.iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+
+/**
+ * Creates an Iterable of Records with all snapshot metadata that can be used with the RecordReader.
+ */
+public class SnapshotIterable implements CloseableIterable {

Review comment:
       My concern is that we will need N such classes for the N metadata tables we offer. I would suggest looking at how this is handled for Spark.
   
   https://github.com/apache/iceberg/blob/cad124967ceb740178a3f65b75f19017494b430d/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java#L108




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-638975489


   Looks like the current problem is that missing `org.pentaho` Jar again. Maybe we should move that exclusion into the `configurations.all` block so it's global.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440721340



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case BOOLEAN:
+        return leaf.getLiteral();
+      default:
+        throw new IllegalStateException("Unknown type: " + leaf.getType());
+    }
+  }
+
+  private static List<Object> hiveLiteralListToIcebergType(List<Object> hiveLiteralTypes) {

Review comment:
       I would agree about trying to reuse `leafToIcebergType`. Some refactoring needs to happen for the `IN` operator case where we need to convert a literal List instead of just a single literal and my current logic doesn't support that very well - but I am on it!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440696009



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Yeah you're correct about the Date type, and I was using that initially, but then when running the tests I would get a `ClassCastException` as it looks like Hive does some internal conversion into a Timestamp when calling leaf.getLiteral for a Date type: 
   https://github.com/apache/hive/blob/branch-2.3/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L108




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r435400649



##########
File path: mr/dependencies.lock
##########
@@ -1,4 +1,365 @@
 {
+    "allProcessors": {

Review comment:
       This is also added in #1086, so I think it is the result of a change to config that wasn't reflected in the existing lock files. I wouldn't worry about it too much, unless it's causing issues. This configuration should just be internal to Gradle.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439672040



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/TableResolver.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.mapred;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+final class TableResolver {

Review comment:
       👍  I think the overall functionality offered by both classes is the same. Although there is some feature parity at the moment w.r.t. metadata tables. I guess we can revisit this at a later point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440703210



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Also, might be good to mention this Hive behaviour as a comment as it looks very non-intuitive.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r441454383



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {

Review comment:
       Thanks for the suggestion, done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-638370785


   An update on progress here - we've been blocked waiting on changes in master that allow subprojects to specify their own versions of dependencies. The Nebula versions plugin (#1067) has been merged into master which allows just that. I have merged master into this branch and next up we'll add a HiveRunner unit test that demonstrates the Hive InputFormat working end to end. We have some other bug fixes for the InputFormat that we need to merge in and then hopefully this will be ready for a proper review.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440733153



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();
+      case DECIMAL:
+        return BigDecimal.valueOf(((HiveDecimalWritable) leaf.getLiteral()).doubleValue());
+      case TIMESTAMP:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       ah nice, thank you for the pointer! 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439643514



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/iterables/SnapshotIterable.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.mapred.iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+
+/**
+ * Creates an Iterable of Records with all snapshot metadata that can be used with the RecordReader.
+ */
+public class SnapshotIterable implements CloseableIterable {

Review comment:
       Also we can probably consider adding support for metadata tables in a separate PR? This PR has a lot of new functionality to review.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.mapred;
+
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {
+  }
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    List<PredicateLeaf> leaves = sarg.getLeaves();
+    List<ExpressionTree> childNodes = sarg.getExpression().getChildren();
+
+    switch (sarg.getExpression().getOperator()) {
+      case OR:
+        ExpressionTree orLeft = childNodes.get(0);
+        ExpressionTree orRight = childNodes.get(1);
+        return or(translate(orLeft, leaves), translate(orRight, leaves));
+      case AND:
+        ExpressionTree andLeft = childNodes.get(0);
+        ExpressionTree andRight = childNodes.get(1);
+        if (childNodes.size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(childNodes, leaves);
+          return and(
+                  translate(andLeft, leaves), translate(andRight, leaves), evaluatedChildren);
+        } else {
+          return and(translate(andLeft, leaves), translate(andRight, leaves));
+        }
+      case NOT:
+        return not(translateLeaf(sarg.getLeaves().get(0)));
+      case LEAF:
+        return translateLeaf(sarg.getLeaves().get(0));
+      case CONSTANT:
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + sarg.getExpression().getOperator());
+    }
+  }
+
+  /**
+   * Remove first 2 nodes already evaluated and return an array of the evaluated leftover nodes.
+   * @param allChildNodes All child nodes to be evaluated for the AND expression.
+   * @param leaves All instances of the leaf nodes.
+   * @return Array of leftover evaluated nodes.
+   */
+  private static Expression[] getLeftoverLeaves(List<ExpressionTree> allChildNodes, List<PredicateLeaf> leaves) {
+    allChildNodes.remove(0);
+    allChildNodes.remove(0);
+
+    Expression[] evaluatedLeaves = new Expression[allChildNodes.size()];
+    for (int i = 0; i < allChildNodes.size(); i++) {
+      Expression filter = translate(allChildNodes.get(i), leaves);
+      evaluatedLeaves[i] = filter;
+    }
+    return evaluatedLeaves;
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    switch (tree.getOperator()) {
+      case OR:
+        return or(translate(tree.getChildren().get(0), leaves),
+                translate(tree.getChildren().get(1), leaves));
+      case AND:
+        if (tree.getChildren().size() > 2) {
+          Expression[] evaluatedChildren = getLeftoverLeaves(tree.getChildren(), leaves);
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves), evaluatedChildren);
+        } else {
+          return and(translate(tree.getChildren().get(0), leaves),
+                  translate(tree.getChildren().get(1), leaves));
+        }
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        return null;
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    if (column.equals("snapshot__id")) {
+      return Expressions.alwaysTrue();
+    }
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leaf.getLiteral());

Review comment:
       You can also take some insights from https://github.com/apache/iceberg/blob/cad124967ceb740178a3f65b75f19017494b430d/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java#L241 which is converting the other way round. This deals with ORC's SearchArgument but it is pretty much the same as Hive's SearchArgument.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440070887



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -131,7 +90,7 @@ private static Expression translate(ExpressionTree tree, List<PredicateLeaf> lea
    */
   private static Expression translateLeaf(PredicateLeaf leaf) {
     String column = leaf.getColumnName();
-    if (column.equals("snapshot__id")) {
+    if (column.equals(SystemTableUtil.DEFAULT_SNAPSHOT_ID_COLUMN_NAME)) {

Review comment:
       A good point, I'll address this in the follow up PR




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439544211



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/iterables/SnapshotIterable.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.mapred.iterables;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+
+/**
+ * Creates an Iterable of Records with all snapshot metadata that can be used with the RecordReader.
+ */
+public class SnapshotIterable implements CloseableIterable {

Review comment:
       The reason we ended up adding the `SnapshotIterable` is so we could wrap a row from the SnapshotsTable in a `Record` which could be passed back to the `RecordReader` without needing to change any code for the RR. It could be super helpful if there was a method in `SnapshotsTable` that returned Records for the whole table, as there isn't a Reader, so to speak, for the `METADATA` file format (https://github.com/apache/iceberg/blob/master/api/src/main/java/org/apache/iceberg/FileFormat.java#L31) that we can add in to the `ReaderFactory` that would allow us 'read' from the SnapshotTable




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r439456480



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/TableResolver.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.mapred;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Properties;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.mr.InputFormatConfig;
+
+final class TableResolver {

Review comment:
       The intention was to not have a class that has both the `mapred` and the `mapreduce` apis as imports just to keep the APIs completely separate and prevent mistakes. If we feel it would be beneficial to combine them into one and it's still simple to see which paths are used by which APIs (where they don't overlap) then I guess we could combine them. As it stands right now I'm not sure there is enough overlapping code to justify doing this?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/iceberg/pull/933#discussion_r440696009



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergFilterFactory.java
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.mapred;
+
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.List;
+import org.apache.hadoop.hive.ql.io.sarg.ExpressionTree;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
+import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+
+public class IcebergFilterFactory {
+
+  private IcebergFilterFactory() {}
+
+  public static Expression generateFilterExpression(SearchArgument sarg) {
+    return translate(sarg.getExpression(), sarg.getLeaves());
+  }
+
+  /**
+   * Recursive method to traverse down the ExpressionTree to evaluate each expression and its leaf nodes.
+   * @param tree Current ExpressionTree where the 'top' node is being evaluated.
+   * @param leaves List of all leaf nodes within the tree.
+   * @return Expression that is translated from the Hive SearchArgument.
+   */
+  private static Expression translate(ExpressionTree tree, List<PredicateLeaf> leaves) {
+    List<ExpressionTree> childNodes = tree.getChildren();
+    switch (tree.getOperator()) {
+      case OR:
+        Expression orResult = Expressions.alwaysFalse();
+        for (ExpressionTree child : childNodes) {
+          orResult = or(orResult, translate(child, leaves));
+        }
+        return orResult;
+      case AND:
+        Expression result = Expressions.alwaysTrue();
+        for (ExpressionTree child : childNodes) {
+          result = and(result, translate(child, leaves));
+        }
+        return result;
+      case NOT:
+        return not(translate(tree.getChildren().get(0), leaves));
+      case LEAF:
+        return translateLeaf(leaves.get(tree.getLeaf()));
+      case CONSTANT:
+        //We are unsure of how the CONSTANT case works, so using the approach of:
+        //https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/read/
+        // ParquetFilterPredicateConverter.java#L116
+        throw new UnsupportedOperationException("CONSTANT operator is not supported");
+      default:
+        throw new IllegalStateException("Unknown operator: " + tree.getOperator());
+    }
+  }
+
+  /**
+   * Translate leaf nodes from Hive operator to Iceberg operator.
+   * @param leaf Leaf node
+   * @return Expression fully translated from Hive PredicateLeaf
+   */
+  private static Expression translateLeaf(PredicateLeaf leaf) {
+    String column = leaf.getColumnName();
+    switch (leaf.getOperator()) {
+      case EQUALS:
+        return equal(column, leafToIcebergType(leaf));
+      case NULL_SAFE_EQUALS:
+        return equal(notNull(column).ref().name(), leafToIcebergType(leaf)); //TODO: Unsure..
+      case LESS_THAN:
+        return lessThan(column, leafToIcebergType(leaf));
+      case LESS_THAN_EQUALS:
+        return lessThanOrEqual(column, leafToIcebergType(leaf));
+      case IN:
+        return in(column, hiveLiteralListToIcebergType(leaf.getLiteralList()));
+      case BETWEEN:
+        List<Object> icebergLiterals = hiveLiteralListToIcebergType(leaf.getLiteralList());
+        return and(greaterThanOrEqual(column, icebergLiterals.get(0)),
+            lessThanOrEqual(column, icebergLiterals.get(1)));
+      case IS_NULL:
+        return isNull(column);
+      default:
+        throw new IllegalStateException("Unknown operator: " + leaf.getOperator());
+    }
+  }
+
+  private static Object leafToIcebergType(PredicateLeaf leaf) {
+    switch (leaf.getType()) {
+      case LONG:
+        return leaf.getLiteral();
+      case FLOAT:
+        return leaf.getLiteral();
+      case STRING:
+        return leaf.getLiteral();
+      case DATE:
+        return ((Timestamp) leaf.getLiteral()).getTime();

Review comment:
       Yeah you're correct about the Date type, and I was using that initially, but then when running the tests I would get a `ClassCastException: Can't cast Timestamp as Date`: it looks like Hive does some internal conversion into a Timestamp when calling leaf.getLiteral for a Date type: 
   https://github.com/apache/hive/blob/branch-2.3/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L108




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] cmathiesen commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on pull request #933:
URL: https://github.com/apache/iceberg/pull/933#issuecomment-643394733


   @shardulm94 thank you for the review! I've made some optimisations to the `FilterFactory` and have left a few questions/discussions open :))


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411459955



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       Can `IcebergSplit` extend `v2InputSplit` and implement `v1InputSplit`
   I tried it and Intellij does not complain. Seems like it would compile
   ```
   static class IcebergSplit extends InputSplit implements Writable, org.apache.hadoop.mapred.InputSplit {..
   ```
   If this works, maybe we can have a common InputSplit for both Mrv1 and Mrv2
   
   Also since RecordReader is the most complex bit. I think we should be able to reuse it for Mrv1 RecordReader.
   What if v1Record internally kept a reference to v2RecordReader? .  Seems like we would only need to call `initialize` correctly. For the we can use `org.apache.hadoop.mapred.TaskAttemptContextImpl#TaskAttemptContextImpl(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.TaskAttemptID)`.   For the `InputSplit` parameter, I think we might need to tweak  `mapreduce.IcebergRecordReader` a little, like so
   
   ```
   @Override
       public void initialize(InputSplit split, TaskAttemptContext newContext) {
         initialize((IcebergSplit) split, newContext);
       }
   
       public void initialize(IcebergSplit split, TaskAttemptContext newContext) {
   ```
   
   This will work since `IcebergSplit1 extends both v1 and v2 InputSplit    
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411459955



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       Can `IcebergSplit` extend `v2InputSplit` and implement `v1InputSplit`
   I tried it and Intellij does not complain. Seems like it would compile
   ```
   static class IcebergSplit extends InputSplit implements Writable, org.apache.hadoop.mapred.InputSplit {..
   ```
   If this works, maybe we can have a common InputSplit for both Mrv1 and Mrv2
   
   Also since RecordReader is the most complex bit. I think we should be able to reuse it for Mrv1 RecordReader.
   What if v1RecordReader internally kept a reference to v2RecordReader? .  Seems like we would only need to call `initialize` correctly. For the we can use `org.apache.hadoop.mapred.TaskAttemptContextImpl#TaskAttemptContextImpl(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.TaskAttemptID)`.   For the `InputSplit` parameter, I think we might need to tweak  `mapreduce.IcebergRecordReader` a little, like so
   
   ```
   @Override
       public void initialize(InputSplit split, TaskAttemptContext newContext) {
         initialize((IcebergSplit) split, newContext);
       }
   
       public void initialize(IcebergSplit split, TaskAttemptContext newContext) {
   ```
   
   This will work since `IcebergSplit1 extends both v1 and v2 InputSplit    
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411531371



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";

Review comment:
       I agree, we can either refer to `mapreduce/IcebergInputFormat` or extract them out into a separate class/interface. Strong preference either way?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411534325



##########
File path: mr/src/test/resources/test-table/metadata/v2.metadata.json
##########
@@ -0,0 +1,47 @@
+{

Review comment:
       These were initially added when we were using [HiveRunner](https://github.com/klarna/HiveRunner/) to do a proper end-to-end test with in-memory Hive and MR etc. We've now moved that code over to the `iceberg-hive` subproject another branch so I'm not sure whether the tests left here still require these files, we'll take a look at which ones can be removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411588844



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       Yea I think it would make sense to have a separate module. Any ideas on the name for that?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411433495



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       Instead of extending `AvoidSplitCombintion` which is very hive specific, we could define a `HiveInputFormat` which extends from `mapred.IcebergInputFormat<T>` and also implements `AvoidSplitCombintion`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411535388



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergWritable.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+
+public class IcebergWritable implements Writable {

Review comment:
       The reason we didn't is because it doesn't have any Hive-specific features or imports in it. We only moved things over to `iceberg-hive` if they had Hive imports (until we ended up needing `CombineHiveInputFormat` above which broke the rule :( like you spotted above).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r414587576



##########
File path: mr/src/test/resources/test-table/metadata/v2.metadata.json
##########
@@ -0,0 +1,47 @@
+{

Review comment:
       I completely agree. I'll try make the test look more like the one @rdsr wrote for the other InputFormat.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411558703



##########
File path: mr/src/test/resources/test-table/metadata/v2.metadata.json
##########
@@ -0,0 +1,47 @@
+{

Review comment:
       We generally prefer to build data files and tables in a setup method, that way we don't add binary files to this repository. This is what we do for all of the Spark tests so I'd like to use that pattern here as well.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on issue #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on issue #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#issuecomment-616545220


   @rdsr we'd appreciate your input on this one as to next steps please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411461775



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergWritable.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+
+public class IcebergWritable implements Writable {

Review comment:
       Does it make sense to move this to `iceberg-hive` module` and it can be part of the `HiveInputFormat` as described above




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r422396125



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       Let's keep it where it is for now. We can move it if we need Hive later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411433495



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       Instead of extending `AvoidSplitCombintion` which is very hive specific, we could define a `HiveInputFormat` which extends from `IcebergInputFormat<T>` and also implements `AvoidSplitCombintion`

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {

Review comment:
       We should reuse the already defined one.
   
   I think it would make sense to have a IcebergMRUtil or IcebergMR class which could house all the constants and the findTable method and `org.apache.iceberg.mr.mapreduce.IcebergInputFormat.ConfigBuilder` class.
   
   Any common method should/constructor should rely on `Configuration` which should work for both MRv1 and Mrv2.  
   @massdosage , @rdblue thoughts?
   

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";

Review comment:
       I think it makes sense to reuse the definition.

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       Can `IcebergSplit` extend `v2InputSplit` and implement `v1InputSplit`
   I tried it and Intellij does not complain. Seems like it would compile
   ```
   static class IcebergSplit extends InputSplit implements Writable, org.apache.hadoop.mapred.InputSplit {..
   ```
   If this works, maybe we can have a common InputSplit for both Mrv1 and Mrv2
   
   Also since RecordReader is the most complex bit. I think we should we able to reuse it for Mrv1 RecordReader.
   What if v1Record internally kept a reference to v2RecordReader? .  Seems like we would only need to call `initialize` correctly. For the we can use `org.apache.hadoop.mapred.TaskAttemptContextImpl#TaskAttemptContextImpl(org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.TaskAttemptID)`.   For the `InputSplit` parameter, I think we might need to tweak  `mapreduce.IcebergRecordReader` a little, like so
   
   ```
   @Override
       public void initialize(InputSplit split, TaskAttemptContext newContext) {
         initialize((IcebergSplit) split, newContext);
       }
   
       public void initialize(IcebergSplit split, TaskAttemptContext newContext) {
   ```
   
   This will work since `IcebergSplit1 extends both v1 and v2 InputSplit    
   

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {

Review comment:
       Should this be `T` instead of `IceberbWritable`?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();

Review comment:
       I guess the writable makes sense in `HiveInputFormat` we discussed above?

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergWritable.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+
+public class IcebergWritable implements Writable {

Review comment:
       Does it make sense to move his to `iceberg-hive` module` and it can be part of the `HiveInputFormat` as described above

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       This can potentially help us rely delegate all calls to `mapreduce.IcebergRecordReader`

##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergReaderFactory.java
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.mapred;
+
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.avro.DataReader;
+import org.apache.iceberg.data.orc.GenericOrcReader;
+import org.apache.iceberg.data.parquet.GenericParquetReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+
+class IcebergReaderFactory {

Review comment:
       We might not need this if the above works

##########
File path: mr/src/test/resources/test-table/metadata/v2.metadata.json
##########
@@ -0,0 +1,47 @@
+{

Review comment:
       Why do we need these?

##########
File path: mr/src/test/resources/test-table/metadata/v1.metadata.json
##########
@@ -0,0 +1,31 @@
+{
+  "format-version" : 1,

Review comment:
       We should exclude the binary files above using `.gitignore`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411557755



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergWritable.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import org.apache.hadoop.io.Writable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.Record;
+
+public class IcebergWritable implements Writable {

Review comment:
       Do we need an Iceberg writable? Shouldn't we use existing writables that Hive expects to receive?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411532049



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {

Review comment:
       We can try and see ;)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411555674



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       I doubt anyone would use the mapred format, so I think it makes sense to only add a `HiveInputFormat`.
   
   I'm not sure we want to add it to `iceberg-hive`, unless we don't add many dependencies. Right now, that module is for interacting with the Hive metastore so it is used in Spark and other places. If we don't need additional dependencies then having a `HiveInputFormat` class makes sense. But if we need to make that module pull in more, then we should consider a separate module for Hive integration.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#issuecomment-624723722


   This is ready for another round of review. We've tried to refactor as much of the shared code between the InputFormats as possible but it's still not perfect. What we'd like input on:
   
   1. General feedback on the code layout, suggestions for improvement, any other shared code that could be cleaned up.
   2. What to do with the Hive dependencies that are creeping into the InputFormat? To make it really useful in Hive we'd actually need to add more (for example you can look at our downstream project that we are adding features in before attempting to merge them here to see what a more complete InputFormat for Hive would look like - https://github.com/ExpediaGroup/hiveberg/blob/master/src/main/java/com/expediagroup/hiveberg/IcebergInputFormat.java). For a more useful Hive integration we would also add a SerDe, StorageHandler etc. We could decide to put all of that code in `iceberg-hive` or create a new `hive-exec` subproject. 
   3. Once we've decided that we need to decide how best to share code between what's in `mr` and whether it's worth having a "base" `mapred` InputFormat in `mr`and then possibly an extension of it in the hive/hive-exec subproject which extends it and adds Hive-specific functionality (this will be more complicated than just moving the whole thing there).
   4. And a gentle reminder that none of this will be usable in Hive until #935 has been completed.
   
   Thoughts?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411600541



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       `iceberg-hive-exec`?
   
   Let's make sure we need it. I think this currently adds `compileOnly` dependencies, so we should be okay to put it 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:
users@infra.apache.org



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411532511



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();

Review comment:
       What else could it be for non-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:
users@infra.apache.org



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411556501



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {

Review comment:
       Can the mapred implementation extend the mapreduce implementation? Then we could manage this through inheritance.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411557155



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       Reusing as much as we can would be ideal.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411530153



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       Yeah, true. Do we think anything other than Hive is going to use this InputFormat? If not then we could just move the whole thing over to `iceberg-hive`. Otherwise we can leave it here and make the change you suggest.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411533093



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       I'll give this a go and see if I can get it to work.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r414484766



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       From our integration tests, what actually happens in Hive is that it wraps the InputFormat with a HiveInputFomat which in turn does an `instanceof FileSplit` which is why we had to extend `FileSplit` here otherwise it throws a RuntimeException and fails. See https://github.com/apache/hive/blob/rel/release-2.3.7/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java#L134. So I don't think we can have a single InputSplit class. I will look into if there is some other way to share as much of the RecordReader code as possible.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r414474126



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       OK, I'm a bit confused now. Do we want to keep this here, move it to `iceberg-hive` or create a new subproject? Bear in mind that there will be other classes like the Hive SerDe, StorageHandler etc. built around this in future. We were intending to put all that in `iceberg-hive` but if you'd prefer to have some other subproject that contains them all that's fine too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411531915



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {

Review comment:
       Sounds like a reasonable approach to me (and answers my question about the constants above).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411437222



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {

Review comment:
       We should reuse the already defined one.
   
   I think it would make sense to have a IcebergMRUtil  class which could house all the constants and the findTable method and `org.apache.iceberg.mr.mapreduce.IcebergInputFormat.ConfigBuilder` class.
   
   Any common method should/constructor should rely on `Configuration` which should work for both MRv1 and Mrv2.  
   @massdosage , @rdblue thoughts?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r411460518



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {
+    HadoopTables tables = new HadoopTables(conf);
+    String tableDir = conf.get("location");
+    if (tableDir == null) {
+      throw new IllegalArgumentException("Table 'location' not set in JobConf");
+    }
+    URI location = null;
+    try {
+      location = new URI(tableDir);
+    } catch (URISyntaxException e) {
+      throw new IOException("Unable to create URI for table location: '" + tableDir + "'", e);
+    }
+    table = tables.load(location.getPath());
+    return table;
+  }
+
+  private InputSplit[] createSplits(List<CombinedScanTask> tasks, String location) {
+    InputSplit[] splits = new InputSplit[tasks.size()];
+    for (int i = 0; i < tasks.size(); i++) {
+      splits[i] = new IcebergSplit(tasks.get(i), location);
+    }
+    return splits;
+  }
+
+  @Override
+  public RecordReader getRecordReader(InputSplit split, JobConf job, Reporter reporter) throws IOException {
+    return new IcebergRecordReader(split, job);
+  }
+
+  @Override
+  public boolean shouldSkipCombine(Path path, Configuration conf) throws IOException {
+    return true;
+  }
+
+  public class IcebergRecordReader implements RecordReader<Void, IcebergWritable> {
+    private JobConf conf;
+    private IcebergSplit split;
+
+    private Iterator<FileScanTask> tasks;
+    private CloseableIterable<Record> reader;
+    private Iterator<Record> recordIterator;
+    private Record currentRecord;
+    private boolean reuseContainers;
+
+    public IcebergRecordReader(InputSplit split, JobConf conf) throws IOException {
+      this.split = (IcebergSplit) split;
+      this.conf = conf;
+      this.reuseContainers = conf.getBoolean(REUSE_CONTAINERS, false);
+      initialise();
+    }
+
+    private void initialise() {
+      tasks = split.getTask().files().iterator();
+      nextTask();
+    }
+
+    private void nextTask() {
+      FileScanTask currentTask = tasks.next();
+      DataFile file = currentTask.file();
+      InputFile inputFile = HadoopInputFile.fromLocation(file.path(), conf);
+      Schema tableSchema = table.schema();
+
+      reader = IcebergReaderFactory.createReader(file, currentTask, inputFile, tableSchema, reuseContainers);
+      recordIterator = reader.iterator();
+    }
+
+    @Override
+    public boolean next(Void key, IcebergWritable value) {
+      if (recordIterator.hasNext()) {
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+
+      if (tasks.hasNext()) {
+        try {
+          reader.close();
+        } catch (IOException e) {
+          LOG.error("Error closing reader", e);
+        }
+        nextTask();
+        currentRecord = recordIterator.next();
+        value.setRecord(currentRecord);
+        return true;
+      }
+      return false;
+    }
+
+    @Override
+    public Void createKey() {
+      return null;
+    }
+
+    @Override
+    public IcebergWritable createValue() {
+      IcebergWritable record = new IcebergWritable();
+      record.setRecord(currentRecord);
+      record.setSchema(table.schema());
+      return record;
+    }
+
+    @Override
+    public long getPos() throws IOException {
+      return 0;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+  }
+
+  /**
+   * FileSplit is extended rather than implementing the InputSplit interface due to Hive's HiveInputFormat
+   * expecting a split which is an instance of FileSplit.
+   */
+  private static class IcebergSplit extends FileSplit {

Review comment:
       This can potentially help us delegate all calls to `mapreduce.IcebergRecordReader`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] cmathiesen commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
cmathiesen commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r416732251



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {

Review comment:
       I think a `HiveInputFormat` is a good idea, as we've been using a lot of Hive specific stuff to get this IF working with Hive (things like the interface @rdsr mentioned above and some properties in the JobConf set by Hive for things like column projection)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [incubator-iceberg] massdosage commented on a change in pull request #933: [WIP] Mapred input format

Posted by GitBox <gi...@apache.org>.
massdosage commented on a change in pull request #933:
URL: https://github.com/apache/incubator-iceberg/pull/933#discussion_r414479378



##########
File path: mr/src/main/java/org/apache/iceberg/mr/mapred/IcebergInputFormat.java
##########
@@ -0,0 +1,259 @@
+/*
+ * 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.mapred;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.CombineHiveInputFormat;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mr.SerializationUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * CombineHiveInputFormat.AvoidSplitCombination is implemented to correctly delegate InputSplit
+ * creation to this class. See: https://stackoverflow.com/questions/29133275/
+ * custom-inputformat-getsplits-never-called-in-hive
+ */
+public class IcebergInputFormat<T> implements InputFormat<Void, T>, CombineHiveInputFormat.AvoidSplitCombination {
+  private static final Logger LOG = LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String REUSE_CONTAINERS = "iceberg.mr.reuse.containers";
+
+  private Table table;
+
+  @Override
+  public InputSplit[] getSplits(JobConf conf, int numSplits) throws IOException {
+    table = findTable(conf);
+    CloseableIterable taskIterable = table.newScan().planTasks();
+    List<CombinedScanTask> tasks = (List<CombinedScanTask>) StreamSupport
+        .stream(taskIterable.spliterator(), false)
+        .collect(Collectors.toList());
+    return createSplits(tasks, table.location());
+  }
+
+  private Table findTable(JobConf conf) throws IOException {

Review comment:
       I think extending the other InputFormat would get really confusing. In this class you would then effectively end up with methods like `public InputSplit[] getSplits(JobConf conf, int numSplits` (from the mapred API returning a mapred InputSplit) and then also `public List<InputSplit> getSplits(JobContext context)` (from the mapreduce API returning a mapreduce InputSplit but only the former actually being used. I also worry that there will be some bizarre code in Hadoop or elsewhere that does `instanceof` checks on these classes to determine certain code paths and that could go very wrong if the mapred inputform extends the mapreduce one. 
   
   I would favour moving the duplicated code out more along the lines of what @rdsr  suggests and solve this via composition rather than inheritance.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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