Skip to content

Commit

Permalink
Move Hive ORC reader configs to OrcReaderConfig
Browse files Browse the repository at this point in the history
  • Loading branch information
dain committed Sep 28, 2019
1 parent ba21d89 commit 88dd641
Show file tree
Hide file tree
Showing 11 changed files with 242 additions and 153 deletions.
115 changes: 0 additions & 115 deletions presto-hive/src/main/java/io/prestosql/plugin/hive/HiveConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -88,15 +88,6 @@ public class HiveConfig

private boolean assumeCanonicalPartitionKeys;

private boolean useOrcColumnNames;
private boolean orcBloomFiltersEnabled;
private DataSize orcMaxMergeDistance = new DataSize(1, MEGABYTE);
private DataSize orcMaxBufferSize = new DataSize(8, MEGABYTE);
private DataSize orcTinyStripeThreshold = new DataSize(8, MEGABYTE);
private DataSize orcStreamBufferSize = new DataSize(8, MEGABYTE);
private DataSize orcMaxReadBlockSize = new DataSize(16, MEGABYTE);
private boolean orcLazyReadSmallRanges = true;

private boolean rcfileWriterValidate;

private boolean skipDeletionForAlter;
Expand Down Expand Up @@ -492,112 +483,6 @@ public HiveConfig setWriteValidationThreads(int writeValidationThreads)
return this;
}

public boolean isUseOrcColumnNames()
{
return useOrcColumnNames;
}

@Config("hive.orc.use-column-names")
@ConfigDescription("Access ORC columns using names from the file")
public HiveConfig setUseOrcColumnNames(boolean useOrcColumnNames)
{
this.useOrcColumnNames = useOrcColumnNames;
return this;
}

@NotNull
public DataSize getOrcMaxMergeDistance()
{
return orcMaxMergeDistance;
}

@Config("hive.orc.max-merge-distance")
public HiveConfig setOrcMaxMergeDistance(DataSize orcMaxMergeDistance)
{
this.orcMaxMergeDistance = orcMaxMergeDistance;
return this;
}

@NotNull
public DataSize getOrcMaxBufferSize()
{
return orcMaxBufferSize;
}

@Config("hive.orc.max-buffer-size")
public HiveConfig setOrcMaxBufferSize(DataSize orcMaxBufferSize)
{
this.orcMaxBufferSize = orcMaxBufferSize;
return this;
}

@NotNull
public DataSize getOrcStreamBufferSize()
{
return orcStreamBufferSize;
}

@Config("hive.orc.stream-buffer-size")
public HiveConfig setOrcStreamBufferSize(DataSize orcStreamBufferSize)
{
this.orcStreamBufferSize = orcStreamBufferSize;
return this;
}

@NotNull
public DataSize getOrcTinyStripeThreshold()
{
return orcTinyStripeThreshold;
}

@Config("hive.orc.tiny-stripe-threshold")
public HiveConfig setOrcTinyStripeThreshold(DataSize orcTinyStripeThreshold)
{
this.orcTinyStripeThreshold = orcTinyStripeThreshold;
return this;
}

@NotNull
public DataSize getOrcMaxReadBlockSize()
{
return orcMaxReadBlockSize;
}

@Config("hive.orc.max-read-block-size")
public HiveConfig setOrcMaxReadBlockSize(DataSize orcMaxReadBlockSize)
{
this.orcMaxReadBlockSize = orcMaxReadBlockSize;
return this;
}

@Deprecated
public boolean isOrcLazyReadSmallRanges()
{
return orcLazyReadSmallRanges;
}

// TODO remove config option once efficacy is proven
@Deprecated
@Config("hive.orc.lazy-read-small-ranges")
@ConfigDescription("ORC read small disk ranges lazily")
public HiveConfig setOrcLazyReadSmallRanges(boolean orcLazyReadSmallRanges)
{
this.orcLazyReadSmallRanges = orcLazyReadSmallRanges;
return this;
}

public boolean isOrcBloomFiltersEnabled()
{
return orcBloomFiltersEnabled;
}

@Config("hive.orc.bloom-filters.enabled")
public HiveConfig setOrcBloomFiltersEnabled(boolean orcBloomFiltersEnabled)
{
this.orcBloomFiltersEnabled = orcBloomFiltersEnabled;
return this;
}

public boolean isRcfileWriterValidate()
{
return rcfileWriterValidate;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import io.prestosql.plugin.hive.metastore.SemiTransactionalHiveMetastore;
import io.prestosql.plugin.hive.orc.OrcFileWriterFactory;
import io.prestosql.plugin.hive.orc.OrcPageSourceFactory;
import io.prestosql.plugin.hive.orc.OrcReaderConfig;
import io.prestosql.plugin.hive.orc.OrcWriterConfig;
import io.prestosql.plugin.hive.parquet.ParquetPageSourceFactory;
import io.prestosql.plugin.hive.parquet.ParquetReaderConfig;
Expand Down Expand Up @@ -109,6 +110,7 @@ public void configure(Binder binder)
Multibinder<HiveFileWriterFactory> fileWriterFactoryBinder = newSetBinder(binder, HiveFileWriterFactory.class);
binder.bind(OrcFileWriterFactory.class).in(Scopes.SINGLETON);
newExporter(binder).export(OrcFileWriterFactory.class).withGeneratedName();
configBinder(binder).bindConfig(OrcReaderConfig.class);
configBinder(binder).bindConfig(OrcWriterConfig.class);
fileWriterFactoryBinder.addBinding().to(OrcFileWriterFactory.class).in(Scopes.SINGLETON);
fileWriterFactoryBinder.addBinding().to(RcFileFileWriterFactory.class).in(Scopes.SINGLETON);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
import com.google.common.collect.ImmutableList;
import io.airlift.units.DataSize;
import io.prestosql.orc.OrcWriteValidation.OrcWriteValidationMode;
import io.prestosql.plugin.hive.orc.OrcReaderConfig;
import io.prestosql.plugin.hive.orc.OrcWriterConfig;
import io.prestosql.plugin.hive.parquet.ParquetReaderConfig;
import io.prestosql.plugin.hive.parquet.ParquetWriterConfig;
Expand Down Expand Up @@ -107,6 +108,7 @@ public static InsertExistingPartitionsBehavior valueOf(String value, boolean imm
@Inject
public HiveSessionProperties(
HiveConfig hiveConfig,
OrcReaderConfig orcReaderConfig,
OrcWriterConfig orcWriterConfig,
ParquetReaderConfig parquetReaderConfig,
ParquetWriterConfig parquetWriterConfig)
Expand Down Expand Up @@ -134,37 +136,37 @@ public HiveSessionProperties(
booleanProperty(
ORC_BLOOM_FILTERS_ENABLED,
"ORC: Enable bloom filters for predicate pushdown",
hiveConfig.isOrcBloomFiltersEnabled(),
orcReaderConfig.isBloomFiltersEnabled(),
false),
dataSizeProperty(
ORC_MAX_MERGE_DISTANCE,
"ORC: Maximum size of gap between two reads to merge into a single read",
hiveConfig.getOrcMaxMergeDistance(),
orcReaderConfig.getMaxMergeDistance(),
false),
dataSizeProperty(
ORC_MAX_BUFFER_SIZE,
"ORC: Maximum size of a single read",
hiveConfig.getOrcMaxBufferSize(),
orcReaderConfig.getMaxBufferSize(),
false),
dataSizeProperty(
ORC_STREAM_BUFFER_SIZE,
"ORC: Size of buffer for streaming reads",
hiveConfig.getOrcStreamBufferSize(),
orcReaderConfig.getStreamBufferSize(),
false),
dataSizeProperty(
ORC_TINY_STRIPE_THRESHOLD,
"ORC: Threshold below which an ORC stripe or file will read in its entirety",
hiveConfig.getOrcTinyStripeThreshold(),
orcReaderConfig.getTinyStripeThreshold(),
false),
dataSizeProperty(
ORC_MAX_READ_BLOCK_SIZE,
"ORC: Soft max size of Presto blocks produced by ORC reader",
hiveConfig.getOrcMaxReadBlockSize(),
orcReaderConfig.getMaxBlockSize(),
false),
booleanProperty(
ORC_LAZY_READ_SMALL_RANGES,
"Experimental: ORC: Read small file segments lazily",
hiveConfig.isOrcLazyReadSmallRanges(),
orcReaderConfig.isLazyReadSmallRanges(),
false),
dataSizeProperty(
ORC_STRING_STATISTICS_LIMIT,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import io.prestosql.plugin.hive.FileFormatDataSourceStats;
import io.prestosql.plugin.hive.HdfsEnvironment;
import io.prestosql.plugin.hive.HiveColumnHandle;
import io.prestosql.plugin.hive.HiveConfig;
import io.prestosql.plugin.hive.HivePageSourceFactory;
import io.prestosql.spi.PrestoException;
import io.prestosql.spi.connector.ConnectorPageSource;
Expand Down Expand Up @@ -83,9 +82,9 @@ public class OrcPageSourceFactory
private final FileFormatDataSourceStats stats;

@Inject
public OrcPageSourceFactory(TypeManager typeManager, HiveConfig config, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats)
public OrcPageSourceFactory(TypeManager typeManager, OrcReaderConfig config, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats)
{
this(typeManager, requireNonNull(config, "config is null").isUseOrcColumnNames(), hdfsEnvironment, stats);
this(typeManager, requireNonNull(config, "config is null").isUseColumnNames(), hdfsEnvironment, stats);
}

public OrcPageSourceFactory(TypeManager typeManager, boolean useOrcColumnNames, HdfsEnvironment hdfsEnvironment, FileFormatDataSourceStats stats)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
/*
* Licensed 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 io.prestosql.plugin.hive.orc;

import io.airlift.configuration.Config;
import io.airlift.configuration.ConfigDescription;
import io.airlift.units.DataSize;

import javax.validation.constraints.NotNull;

import static io.airlift.units.DataSize.Unit.MEGABYTE;

public class OrcReaderConfig
{
private boolean useColumnNames;
private boolean bloomFiltersEnabled;

private DataSize maxMergeDistance = new DataSize(1, MEGABYTE);
private DataSize maxBufferSize = new DataSize(8, MEGABYTE);
private DataSize tinyStripeThreshold = new DataSize(8, MEGABYTE);
private DataSize streamBufferSize = new DataSize(8, MEGABYTE);
private DataSize maxBlockSize = new DataSize(16, MEGABYTE);
private boolean lazyReadSmallRanges = true;

public boolean isUseColumnNames()
{
return useColumnNames;
}

@Config("hive.orc.use-column-names")
@ConfigDescription("Access ORC columns using names from the file")
public OrcReaderConfig setUseColumnNames(boolean useColumnNames)
{
this.useColumnNames = useColumnNames;
return this;
}

public boolean isBloomFiltersEnabled()
{
return bloomFiltersEnabled;
}

@Config("hive.orc.bloom-filters.enabled")
public OrcReaderConfig setBloomFiltersEnabled(boolean bloomFiltersEnabled)
{
this.bloomFiltersEnabled = bloomFiltersEnabled;
return this;
}

@NotNull
public DataSize getMaxMergeDistance()
{
return maxMergeDistance;
}

@Config("hive.orc.max-merge-distance")
public OrcReaderConfig setMaxMergeDistance(DataSize maxMergeDistance)
{
this.maxMergeDistance = maxMergeDistance;
return this;
}

@NotNull
public DataSize getMaxBufferSize()
{
return maxBufferSize;
}

@Config("hive.orc.max-buffer-size")
public OrcReaderConfig setMaxBufferSize(DataSize maxBufferSize)
{
this.maxBufferSize = maxBufferSize;
return this;
}

@NotNull
public DataSize getTinyStripeThreshold()
{
return tinyStripeThreshold;
}

@Config("hive.orc.tiny-stripe-threshold")
public OrcReaderConfig setTinyStripeThreshold(DataSize tinyStripeThreshold)
{
this.tinyStripeThreshold = tinyStripeThreshold;
return this;
}

@NotNull
public DataSize getStreamBufferSize()
{
return streamBufferSize;
}

@Config("hive.orc.stream-buffer-size")
public OrcReaderConfig setStreamBufferSize(DataSize streamBufferSize)
{
this.streamBufferSize = streamBufferSize;
return this;
}

@NotNull
public DataSize getMaxBlockSize()
{
return maxBlockSize;
}

@Config("hive.orc.max-read-block-size")
public OrcReaderConfig setMaxBlockSize(DataSize maxBlockSize)
{
this.maxBlockSize = maxBlockSize;
return this;
}

@Deprecated
public boolean isLazyReadSmallRanges()
{
return lazyReadSmallRanges;
}

// TODO remove config option once efficacy is proven
@Deprecated
@Config("hive.orc.lazy-read-small-ranges")
@ConfigDescription("ORC read small disk ranges lazily")
public OrcReaderConfig setLazyReadSmallRanges(boolean lazyReadSmallRanges)
{
this.lazyReadSmallRanges = lazyReadSmallRanges;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import io.prestosql.plugin.hive.gcs.HiveGcsConfig;
import io.prestosql.plugin.hive.orc.OrcFileWriterFactory;
import io.prestosql.plugin.hive.orc.OrcPageSourceFactory;
import io.prestosql.plugin.hive.orc.OrcReaderConfig;
import io.prestosql.plugin.hive.orc.OrcWriterConfig;
import io.prestosql.plugin.hive.parquet.ParquetPageSourceFactory;
import io.prestosql.plugin.hive.parquet.ParquetReaderConfig;
Expand Down Expand Up @@ -81,6 +82,7 @@ public static HiveSessionProperties getHiveSessionProperties(HiveConfig hiveConf
{
return new HiveSessionProperties(
hiveConfig,
new OrcReaderConfig(),
new OrcWriterConfig(),
new ParquetReaderConfig(),
new ParquetWriterConfig());
Expand All @@ -91,7 +93,7 @@ public static Set<HivePageSourceFactory> getDefaultHivePageSourceFactories(HiveC
FileFormatDataSourceStats stats = new FileFormatDataSourceStats();
return ImmutableSet.<HivePageSourceFactory>builder()
.add(new RcFilePageSourceFactory(TYPE_MANAGER, hdfsEnvironment, stats))
.add(new OrcPageSourceFactory(TYPE_MANAGER, hiveConfig, hdfsEnvironment, stats))
.add(new OrcPageSourceFactory(TYPE_MANAGER, new OrcReaderConfig(), hdfsEnvironment, stats))
.add(new ParquetPageSourceFactory(TYPE_MANAGER, hdfsEnvironment, stats))
.build();
}
Expand Down
Loading

0 comments on commit 88dd641

Please sign in to comment.