diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java index a53f35414ebed..58c0b83cca059 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/HiveUtil.java @@ -20,6 +20,7 @@ import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeManager; import com.google.common.base.Function; +import com.google.common.base.Joiner; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import io.airlift.slice.Slice; @@ -31,7 +32,6 @@ import org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat; import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.SerDeException; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -89,6 +89,8 @@ import static org.apache.hadoop.hive.metastore.MetaStoreUtils.getTableMetadata; import static org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.FILE_INPUT_FORMAT; import static org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_LIB; +import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_ALL_COLUMNS; +import static org.apache.hadoop.hive.serde2.ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR; import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; public final class HiveUtil @@ -124,7 +126,7 @@ private HiveUtil() List readHiveColumnIndexes = ImmutableList.copyOf(transform(readColumns, hiveColumnIndexGetter())); // Tell hive the columns we would like to read, this lets hive optimize reading column oriented files - ColumnProjectionUtils.appendReadColumns(configuration, readHiveColumnIndexes); + setReadColumns(configuration, readHiveColumnIndexes); final InputFormat inputFormat = getInputFormat(configuration, schema, true); final JobConf jobConf = new JobConf(configuration); @@ -159,6 +161,12 @@ private HiveUtil() } } + public static void setReadColumns(Configuration configuration, List readHiveColumnIndexes) + { + configuration.set(READ_COLUMN_IDS_CONF_STR, Joiner.on(',').join(readHiveColumnIndexes)); + configuration.setBoolean(READ_ALL_COLUMNS, false); + } + static InputFormat getInputFormat(Configuration configuration, Properties schema, boolean symlinkTarget) { String inputFormatName = getInputFormatName(schema); diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java index 113f4f99ad561..de18844f5b9a4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/rcfile/RcFilePageSourceFactory.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.ql.io.RCFile; -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; import org.apache.hadoop.hive.serde2.Deserializer; import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; import org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe; @@ -43,6 +42,7 @@ import static com.facebook.presto.hive.HiveColumnHandle.isPartitionKeyPredicate; import static com.facebook.presto.hive.HiveSessionProperties.isOptimizedReaderEnabled; import static com.facebook.presto.hive.HiveUtil.getDeserializer; +import static com.facebook.presto.hive.HiveUtil.setReadColumns; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Predicates.not; import static com.google.common.collect.Iterables.filter; @@ -109,7 +109,7 @@ else if (deserializer instanceof ColumnarSerDe) { List readHiveColumnIndexes = ImmutableList.copyOf(transform(readColumns, hiveColumnIndexGetter())); // Tell hive the columns we would like to read, this lets hive optimize reading column oriented files - ColumnProjectionUtils.appendReadColumns(configuration, readHiveColumnIndexes); + setReadColumns(configuration, readHiveColumnIndexes); // propagate serialization configuration to getRecordReader for (String name : schema.stringPropertyNames()) {