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

 ##########
 File path: mr/src/main/java/org/apache/iceberg/mr/IcebergInputFormat.java
 ##########
 @@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.mr;
+
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.avro.Avro;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.common.DynClasses;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.hadoop.HadoopInputFile;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.hive.HiveCatalogs;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.orc.ORC;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.types.TypeUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class IcebergInputFormat<T> extends InputFormat<Void, T> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(IcebergInputFormat.class);
+
+  static final String FILTER_EXPRESSION = "iceberg.mr.filter.expression";
+  static final String TABLE_SCHEMA = "iceberg.mr.table.schema";
+  static final String TABLE_PATH = "iceberg.mr.table.path";
+  static final String READ_SCHEMA = "iceberg.mr.read.schema";
+  static final String READ_SUPPORT = "iceberg.mr.read.support";
+
+  private transient Table table;
+  private transient List<InputSplit> splits;
+
+  public IcebergInputFormat() {
+  }
+
+  @Override
+  public List<InputSplit> getSplits(JobContext context) {
+    if (splits != null) {
+      LOG.info("Returning cached splits: {}", splits.size());
+      return splits;
+    }
+
+    Configuration conf = context.getConfiguration();
+    table = getTable(conf);
+    TableScan scan = table.newScan();
+    //TODO add caseSensitive, snapshot id etc..
+
+    Expression filterExpression = 
SerializationUtil.deserializeFromBase64(conf.get(FILTER_EXPRESSION));
+    if (filterExpression != null) {
+      scan = scan.filter(filterExpression);
+    }
+
+    final String schemaStr = conf.get(READ_SCHEMA);
+    if (schemaStr != null) {
+      // Not sure if this is having any effect?
+      scan.project(SchemaParser.fromJson(schemaStr));
+    }
+
+    splits = Lists.newArrayList();
+    try (CloseableIterable<CombinedScanTask> tasksIterable = scan.planTasks()) 
{
+      tasksIterable.forEach(task -> splits.add(new IcebergSplit(task)));
+    } catch (IOException e) {
+      throw new RuntimeIOException(e, "Failed to close table scan: %s", scan);
+    }
+
+    return splits;
+  }
+
+  @Override
+  public RecordReader<Void, T> createRecordReader(InputSplit split, 
TaskAttemptContext context) {
+    return new IcebergRecordReader();
+  }
+
+  public static ConfBuilder updateConf(
+      Configuration conf, String path, Class<? extends ReadSupport<?>> 
readSupportClass) {
+    return new ConfBuilder(conf, path, readSupportClass);
+  }
+
+  public static class ConfBuilder {
+    private final Configuration conf;
+
+    public ConfBuilder(Configuration conf, String path, Class<? extends 
ReadSupport<?>> readSupportClass) {
 
 Review comment:
   Decided to set everything as builder methods except configuration, similar 
to Kite

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


With regards,
Apache Git Services

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

Reply via email to