Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,11 @@
import java.util.Arrays;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
import org.apache.hadoop.hive.ql.exec.Utilities;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedInputFormatInterface;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedSupport;
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.SearchArgument;
Expand All @@ -34,20 +38,43 @@
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.RecordReader;
import org.apache.hadoop.mapred.Reporter;
import org.apache.iceberg.common.DynConstructors;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.hive.MetastoreUtil;
import org.apache.iceberg.mr.InputFormatConfig;
import org.apache.iceberg.mr.mapred.AbstractMapredIcebergRecordReader;
import org.apache.iceberg.mr.mapred.Container;
import org.apache.iceberg.mr.mapred.MapredIcebergInputFormat;
import org.apache.iceberg.mr.mapreduce.IcebergInputFormat;
import org.apache.iceberg.mr.mapreduce.IcebergSplit;
import org.apache.iceberg.mr.mapreduce.IcebergSplitContainer;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.util.SerializationUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

public class HiveIcebergInputFormat extends MapredIcebergInputFormat<Record>
implements CombineHiveInputFormat.AvoidSplitCombination {
implements CombineHiveInputFormat.AvoidSplitCombination, VectorizedInputFormatInterface {

private static final Logger LOG = LoggerFactory.getLogger(HiveIcebergInputFormat.class);
private static final String HIVE_VECTORIZED_RECORDREADER_CLASS =
"org.apache.iceberg.mr.hive.vector.HiveIcebergVectorizedRecordReader";
private static final DynConstructors.Ctor<AbstractMapredIcebergRecordReader> HIVE_VECTORIZED_RECORDREADER_CTOR;

static {
if (MetastoreUtil.hive3PresentOnClasspath()) {
HIVE_VECTORIZED_RECORDREADER_CTOR = DynConstructors.builder(AbstractMapredIcebergRecordReader.class)
.impl(HIVE_VECTORIZED_RECORDREADER_CLASS,
IcebergInputFormat.class,
IcebergSplit.class,
JobConf.class,
Reporter.class)
.build();
} else {
HIVE_VECTORIZED_RECORDREADER_CTOR = null;
}
}

@Override
public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
Expand Down Expand Up @@ -77,11 +104,33 @@ public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException {
public RecordReader<Void, Container<Record>> getRecordReader(InputSplit split, JobConf job,
Reporter reporter) throws IOException {
job.set(InputFormatConfig.SELECTED_COLUMNS, job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, ""));
return super.getRecordReader(split, job, reporter);

if (HiveConf.getBoolVar(job, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED) && Utilities.getIsVectorized(job)) {
Preconditions.checkArgument(MetastoreUtil.hive3PresentOnClasspath(), "Vectorization only supported for Hive 3+");

job.setEnum(InputFormatConfig.IN_MEMORY_DATA_MODEL, InputFormatConfig.InMemoryDataModel.HIVE);
job.setBoolean(InputFormatConfig.SKIP_RESIDUAL_FILTERING, true);

IcebergSplit icebergSplit = ((IcebergSplitContainer) split).icebergSplit();
// bogus cast for favouring code reuse over syntax
return (RecordReader) HIVE_VECTORIZED_RECORDREADER_CTOR.newInstance(
new org.apache.iceberg.mr.mapreduce.IcebergInputFormat<>(),
icebergSplit,
job,
reporter);
} else {
return super.getRecordReader(split, job, reporter);
}
}

@Override
public boolean shouldSkipCombine(Path path, Configuration conf) {
return true;
}

@Override
public VectorizedSupport.Support[] getSupportedFeatures() {
return new VectorizedSupport.Support[]{ VectorizedSupport.Support.DECIMAL_64 };
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -91,10 +91,6 @@ public void initialize(@Nullable Configuration configuration, Properties serDePr
// executor, but serDeProperties are populated by HiveIcebergStorageHandler.configureInputJobProperties() and
// the resulting properties are serialized and distributed to the executors

// temporarily disabling vectorization in Tez, since it doesn't work with projection pruning (fix: TEZ-4248)
// TODO: remove this once TEZ-4248 has been released and the Tez dependencies updated here
assertNotVectorizedTez(configuration);

if (serDeProperties.get(InputFormatConfig.TABLE_SCHEMA) != null) {
this.tableSchema = SchemaParser.fromJson((String) serDeProperties.get(InputFormatConfig.TABLE_SCHEMA));
if (serDeProperties.get(InputFormatConfig.PARTITION_SPEC) != null) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.hive.vector;

import java.io.IOException;
import org.apache.hadoop.hive.llap.LlapHiveUtils;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.Reporter;
import org.apache.iceberg.mr.mapred.AbstractMapredIcebergRecordReader;
import org.apache.iceberg.mr.mapreduce.IcebergSplit;

/**
* Basically an MR1 API implementing wrapper for transferring VectorizedRowBatch's produced by
* IcebergInputformat$IcebergRecordReader which relies on the MR2 API format.
*/
public final class HiveIcebergVectorizedRecordReader extends AbstractMapredIcebergRecordReader<VectorizedRowBatch> {

private final JobConf job;

public HiveIcebergVectorizedRecordReader(
org.apache.iceberg.mr.mapreduce.IcebergInputFormat<VectorizedRowBatch> mapreduceInputFormat, IcebergSplit split,
JobConf job, Reporter reporter) throws IOException {
super(mapreduceInputFormat, split, job, reporter);
this.job = job;
}

@Override
public boolean next(Void key, VectorizedRowBatch value) throws IOException {
try {
if (innerReader.nextKeyValue()) {
VectorizedRowBatch newBatch = (VectorizedRowBatch) innerReader.getCurrentValue();
value.cols = newBatch.cols;
value.endOfFile = newBatch.endOfFile;
value.selectedInUse = newBatch.selectedInUse;
value.size = newBatch.size;
return true;
} else {
return false;
}
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
}
}

@Override
public VectorizedRowBatch createValue() {
return LlapHiveUtils.findMapWork(job).getVectorizedRowBatchCtx().createVectorizedRowBatch();
}

@Override
public long getPos() {
return -1;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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.hive.vector;

import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.apache.commons.lang3.ArrayUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
import org.apache.hadoop.hive.ql.io.orc.OrcSplit;
import org.apache.hadoop.hive.ql.io.orc.VectorizedOrcInputFormat;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
import org.apache.hadoop.io.NullWritable;
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.hadoop.mapreduce.TaskAttemptContext;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.mr.mapred.MapredIcebergInputFormat;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;

/**
* Utility class to create vectorized readers for Hive.
* As per the file format of the task, it will create a matching vectorized record reader that is already implemented
* in Hive. It will also do some tweaks on the produced vectors for Iceberg's use e.g. partition column handling.
*/
public class HiveVectorizedReader {


private HiveVectorizedReader() {

}

public static <D> CloseableIterable<D> reader(InputFile inputFile, FileScanTask task, Map<Integer, ?> idToConstant,
TaskAttemptContext context) {
JobConf job = (JobConf) context.getConfiguration();
Path path = new Path(inputFile.location());
FileFormat format = task.file().format();
Reporter reporter = ((MapredIcebergInputFormat.CompatibilityTaskAttemptContextImpl) context).getLegacyReporter();

// Hive by default requires partition columns to be read too. This is not required for identity partition
// columns, as we will add this as constants later.

int[] partitionColIndices = null;
Object[] partitionValues = null;
PartitionSpec partitionSpec = task.spec();

if (!partitionSpec.isUnpartitioned()) {
List<Integer> readColumnIds = ColumnProjectionUtils.getReadColumnIDs(job);

List<PartitionField> fields = partitionSpec.fields();
List<Integer> partitionColIndicesList = Lists.newLinkedList();
List<Object> partitionValuesList = Lists.newLinkedList();

for (PartitionField field : fields) {
if (field.transform().isIdentity()) {
// Skip reading identity partition columns from source file...
int hiveColIndex = field.sourceId() - 1;
readColumnIds.remove((Integer) hiveColIndex);

// ...and use the corresponding constant value instead
partitionColIndicesList.add(hiveColIndex);
partitionValuesList.add(idToConstant.get(field.sourceId()));
}
}

partitionColIndices = ArrayUtils.toPrimitive(partitionColIndicesList.toArray(new Integer[0]));
partitionValues = partitionValuesList.toArray(new Object[0]);

ColumnProjectionUtils.setReadColumns(job, readColumnIds);
}

try {
switch (format) {
case ORC:
InputSplit split = new OrcSplit(path, null, task.start(), task.length(), (String[]) null, null,
false, false, com.google.common.collect.Lists.newArrayList(), 0, task.length(), path.getParent(), null);
RecordReader<NullWritable, VectorizedRowBatch> recordReader = null;

recordReader = new VectorizedOrcInputFormat().getRecordReader(split, job, reporter);
return createVectorizedRowBatchIterable(recordReader, job, partitionColIndices, partitionValues);

default:
throw new UnsupportedOperationException("Vectorized Hive reading unimplemented for format: " + format);
}

} catch (IOException ioe) {
throw new RuntimeException("Error creating vectorized record reader for " + inputFile, ioe);
}
}

private static <D> CloseableIterable<D> createVectorizedRowBatchIterable(
RecordReader<NullWritable, VectorizedRowBatch> hiveRecordReader, JobConf job, int[] partitionColIndices,
Object[] partitionValues) {

VectorizedRowBatchIterator iterator =
new VectorizedRowBatchIterator(hiveRecordReader, job, partitionColIndices, partitionValues);

return new CloseableIterable<D>() {

@Override
public CloseableIterator iterator() {
return iterator;
}

@Override
public void close() throws IOException {
iterator.close();
}
};
}

}
Loading