Skip to content

Commit

Permalink
Remove unused OrcReader field maxReadSize
Browse files Browse the repository at this point in the history
  • Loading branch information
Ying Su authored and mbasmanova committed Aug 28, 2019
1 parent b2dd996 commit d47b4e4
Show file tree
Hide file tree
Showing 17 changed files with 19 additions and 25 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ public static OrcBatchPageSource createOrcPageSource(

AggregatedMemoryContext systemMemoryUsage = newSimpleAggregatedMemoryContext();
try {
OrcReader reader = new OrcReader(orcDataSource, orcEncoding, maxMergeDistance, maxBufferSize, tinyStripeThreshold, maxReadBlockSize);
OrcReader reader = new OrcReader(orcDataSource, orcEncoding, maxMergeDistance, tinyStripeThreshold, maxReadBlockSize);

List<HiveColumnHandle> physicalColumns = getPhysicalHiveColumnHandles(columns, useOrcColumnNames, reader, path);
ImmutableMap.Builder<Integer, Type> includedColumns = ImmutableMap.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -232,7 +232,7 @@ public static OrcSelectivePageSource createOrcPageSource(

AggregatedMemoryContext systemMemoryUsage = newSimpleAggregatedMemoryContext();
try {
OrcReader reader = new OrcReader(orcDataSource, orcEncoding, maxMergeDistance, maxBufferSize, tinyStripeThreshold, maxReadBlockSize);
OrcReader reader = new OrcReader(orcDataSource, orcEncoding, maxMergeDistance, tinyStripeThreshold, maxReadBlockSize);

checkArgument(!domainPredicate.isNone(), "Unexpected NONE domain");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,6 @@ public TempFileReader(List<Type> types, OrcDataSource dataSource)
ORC,
new DataSize(1, MEGABYTE),
new DataSize(8, MEGABYTE),
new DataSize(8, MEGABYTE),
new DataSize(16, MEGABYTE));

Map<Integer, Type> includedColumns = new HashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,6 @@ public class OrcReader
private final OrcDataSource orcDataSource;
private final ExceptionWrappingMetadataReader metadataReader;
private final DataSize maxMergeDistance;
private final DataSize maxReadSize;
private final DataSize tinyStripeThreshold;
private final DataSize maxBlockSize;
private final HiveWriterVersion hiveWriterVersion;
Expand All @@ -75,17 +74,16 @@ public class OrcReader
private final Optional<OrcWriteValidation> writeValidation;

// This is based on the Apache Hive ORC code
public OrcReader(OrcDataSource orcDataSource, OrcEncoding orcEncoding, DataSize maxMergeDistance, DataSize maxReadSize, DataSize tinyStripeThreshold, DataSize maxBlockSize)
public OrcReader(OrcDataSource orcDataSource, OrcEncoding orcEncoding, DataSize maxMergeDistance, DataSize tinyStripeThreshold, DataSize maxBlockSize)
throws IOException
{
this(orcDataSource, orcEncoding, maxMergeDistance, maxReadSize, tinyStripeThreshold, maxBlockSize, Optional.empty());
this(orcDataSource, orcEncoding, maxMergeDistance, tinyStripeThreshold, maxBlockSize, Optional.empty());
}

OrcReader(
OrcDataSource orcDataSource,
OrcEncoding orcEncoding,
DataSize maxMergeDistance,
DataSize maxReadSize,
DataSize tinyStripeThreshold,
DataSize maxBlockSize,
Optional<OrcWriteValidation> writeValidation)
Expand All @@ -96,7 +94,6 @@ public OrcReader(OrcDataSource orcDataSource, OrcEncoding orcEncoding, DataSize
requireNonNull(orcEncoding, "orcEncoding is null");
this.metadataReader = new ExceptionWrappingMetadataReader(orcDataSource.getId(), orcEncoding.createMetadataReader());
this.maxMergeDistance = requireNonNull(maxMergeDistance, "maxMergeDistance is null");
this.maxReadSize = requireNonNull(maxReadSize, "maxReadSize is null");
this.tinyStripeThreshold = requireNonNull(tinyStripeThreshold, "tinyStripeThreshold is null");
this.maxBlockSize = requireNonNull(maxBlockSize, "maxBlockSize is null");

Expand Down Expand Up @@ -378,7 +375,7 @@ static void validateFile(
readTypes.put(columnIndex, types.get(columnIndex));
}
try {
OrcReader orcReader = new OrcReader(input, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(8, MEGABYTE), new DataSize(8, MEGABYTE), new DataSize(16, MEGABYTE), Optional.of(writeValidation));
OrcReader orcReader = new OrcReader(input, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(8, MEGABYTE), new DataSize(16, MEGABYTE), Optional.of(writeValidation));
try (OrcBatchRecordReader orcRecordReader = orcReader.createBatchRecordReader(readTypes.build(), OrcPredicate.TRUE, hiveStorageTimeZone, newSimpleAggregatedMemoryContext(), INITIAL_BATCH_SIZE)) {
while (orcRecordReader.nextBatch() >= 0) {
// ignored
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,7 @@ OrcBatchRecordReader createRecordReader()
throws IOException
{
OrcDataSource dataSource = new FileOrcDataSource(orcFile, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), true);
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
return orcReader.createBatchRecordReader(
ImmutableMap.of(0, type),
OrcPredicate.TRUE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -162,7 +162,7 @@ public OrcSelectiveRecordReader createRecordReader(Optional<TupleDomainFilter> f
throws IOException
{
OrcDataSource dataSource = new FileOrcDataSource(orcFile, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), true);
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));

return orcReader.createSelectiveRecordReader(
ImmutableMap.of(0, type),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -842,7 +842,7 @@ private static OrcBatchRecordReader createCustomOrcRecordReader(TempFile tempFil
throws IOException
{
OrcDataSource orcDataSource = new FileOrcDataSource(tempFile.getFile(), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), true);
OrcReader orcReader = new OrcReader(orcDataSource, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), MAX_BLOCK_SIZE);
OrcReader orcReader = new OrcReader(orcDataSource, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), MAX_BLOCK_SIZE);

assertEquals(orcReader.getColumnNames(), ImmutableList.of("test"));
assertEquals(orcReader.getFooter().getRowsInRowGroup(), 10_000);
Expand Down Expand Up @@ -901,7 +901,7 @@ static OrcSelectiveRecordReader createCustomOrcSelectiveRecordReader(
throws IOException
{
OrcDataSource orcDataSource = new FileOrcDataSource(tempFile.getFile(), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), true);
OrcReader orcReader = new OrcReader(orcDataSource, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), MAX_BLOCK_SIZE);
OrcReader orcReader = new OrcReader(orcDataSource, orcEncoding, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), MAX_BLOCK_SIZE);

assertEquals(orcReader.getColumnNames(), makeColumnNames(types.size()));
assertEquals(orcReader.getFooter().getRowsInRowGroup(), 10_000);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ public void testIntegration()
public void doIntegration(TestingOrcDataSource orcDataSource, DataSize maxMergeDistance, DataSize maxReadSize, DataSize tinyStripeThreshold)
throws IOException
{
OrcReader orcReader = new OrcReader(orcDataSource, ORC, maxMergeDistance, maxReadSize, tinyStripeThreshold, new DataSize(1, Unit.MEGABYTE));
OrcReader orcReader = new OrcReader(orcDataSource, ORC, maxMergeDistance, tinyStripeThreshold, new DataSize(1, Unit.MEGABYTE));
// 1 for reading file footer
assertEquals(orcDataSource.getReadCount(), 1);
List<StripeInformation> stripes = orcReader.getFooter().getStripes();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -397,7 +397,6 @@ private <K, V> void runTest(String testOrcFileName, Type keyType, Type valueType
OrcEncoding.DWRF,
new DataSize(1, MEGABYTE),
new DataSize(1, MEGABYTE),
new DataSize(1, MEGABYTE),
new DataSize(1, DataSize.Unit.MEGABYTE));
Type mapType = TYPE_MANAGER.getParameterizedType(
StandardTypes.MAP,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public void testReadLz4()
// TODO: use Apache ORC library in OrcTester
byte[] data = toByteArray(getResource("apache-lz4.orc"));

OrcReader orcReader = new OrcReader(new InMemoryOrcDataSource(data), ORC, SIZE, SIZE, SIZE, SIZE);
OrcReader orcReader = new OrcReader(new InMemoryOrcDataSource(data), ORC, SIZE, SIZE, SIZE);

assertEquals(orcReader.getCompressionKind(), LZ4);
assertEquals(orcReader.getFooter().getNumberOfRows(), 10_000);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,7 @@ public void testReadUserMetadata()
createFileWithOnlyUserMetadata(tempFile.getFile(), metadata);

OrcDataSource orcDataSource = new FileOrcDataSource(tempFile.getFile(), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), true);
OrcReader orcReader = new OrcReader(orcDataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(orcDataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
Footer footer = orcReader.getFooter();
Map<String, String> readMetadata = Maps.transformValues(footer.getUserMetadata(), Slice::toStringAscii);
assertEquals(readMetadata, metadata);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ public void testWriteOutputStreamsInOrder()
// read the footer and verify the streams are ordered by size
DataSize dataSize = new DataSize(1, MEGABYTE);
OrcDataSource orcDataSource = new FileOrcDataSource(tempFile.getFile(), dataSize, dataSize, dataSize, true);
Footer footer = new OrcReader(orcDataSource, ORC, dataSize, dataSize, dataSize, dataSize).getFooter();
Footer footer = new OrcReader(orcDataSource, ORC, dataSize, dataSize, dataSize).getFooter();

for (StripeInformation stripe : footer.getStripes()) {
// read the footer
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -263,7 +263,7 @@ private RowBlock read(TempFile tempFile, Type readerType)
{
DataSize dataSize = new DataSize(1, MEGABYTE);
OrcDataSource orcDataSource = new FileOrcDataSource(tempFile.getFile(), dataSize, dataSize, dataSize, true);
OrcReader orcReader = new OrcReader(orcDataSource, ORC, dataSize, dataSize, dataSize, dataSize);
OrcReader orcReader = new OrcReader(orcDataSource, ORC, dataSize, dataSize, dataSize);

Map<Integer, Type> includedColumns = new HashMap<>();
includedColumns.put(0, readerType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,6 @@ public OrcFileInfo rewrite(Map<String, Type> allColumnTypes, Path input, Path ou
orcDataEnvironment.createOrcDataSource(input, readerAttributes),
ORC,
readerAttributes.getMaxMergeDistance(),
readerAttributes.getMaxReadSize(),
readerAttributes.getTinyStripeThreshold(),
HUGE_MAX_READ_BLOCK_SIZE);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -261,7 +261,7 @@ public ConnectorPageSource getPageSource(
AggregatedMemoryContext systemMemoryUsage = newSimpleAggregatedMemoryContext();

try {
OrcReader reader = new OrcReader(dataSource, ORC, readerAttributes.getMaxMergeDistance(), readerAttributes.getMaxReadSize(), readerAttributes.getTinyStripeThreshold(), HUGE_MAX_READ_BLOCK_SIZE);
OrcReader reader = new OrcReader(dataSource, ORC, readerAttributes.getMaxMergeDistance(), readerAttributes.getTinyStripeThreshold(), HUGE_MAX_READ_BLOCK_SIZE);

Map<Long, Integer> indexMap = columnIdIndex(reader.getColumnNames());
ImmutableMap.Builder<Integer, Type> includedColumns = ImmutableMap.builder();
Expand Down Expand Up @@ -411,7 +411,7 @@ private ShardInfo createShardInfo(UUID shardUuid, OptionalInt bucketNumber, Path
private List<ColumnStats> computeShardStats(Path file)
{
try (OrcDataSource dataSource = orcDataEnvironment.createOrcDataSource(file, defaultReaderAttributes)) {
OrcReader reader = new OrcReader(dataSource, ORC, defaultReaderAttributes.getMaxMergeDistance(), defaultReaderAttributes.getMaxReadSize(), defaultReaderAttributes.getTinyStripeThreshold(), HUGE_MAX_READ_BLOCK_SIZE);
OrcReader reader = new OrcReader(dataSource, ORC, defaultReaderAttributes.getMaxMergeDistance(), defaultReaderAttributes.getTinyStripeThreshold(), HUGE_MAX_READ_BLOCK_SIZE);

ImmutableList.Builder<ColumnStats> list = ImmutableList.builder();
for (ColumnInfo info : getColumnInfo(reader)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public static OrcDataSource fileOrcDataSource(File file)
public static OrcBatchRecordReader createReader(OrcDataSource dataSource, List<Long> columnIds, List<Type> types)
throws IOException
{
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));

List<String> columnNames = orcReader.getColumnNames();
assertEquals(columnNames.size(), columnIds.size());
Expand All @@ -77,7 +77,7 @@ public static OrcBatchRecordReader createReader(OrcDataSource dataSource, List<L
public static OrcBatchRecordReader createReaderNoRows(OrcDataSource dataSource)
throws IOException
{
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(dataSource, ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));

assertEquals(orcReader.getColumnNames().size(), 0);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -482,7 +482,7 @@ public void testRewriterDropThenAddDifferentColumns()
assertEquals(info.getRowCount(), 4);

// Optimized writer will keep the only column
OrcReader orcReader = new OrcReader(fileOrcDataSource(newFile2), ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
OrcReader orcReader = new OrcReader(fileOrcDataSource(newFile2), ORC, new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE), new DataSize(1, MEGABYTE));
orcReader.getColumnNames().equals(ImmutableList.of("7"));

// Add a column with the different ID with different type
Expand Down

0 comments on commit d47b4e4

Please sign in to comment.