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
26 changes: 13 additions & 13 deletions build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,8 @@ subprojects {
all {
exclude group: 'org.slf4j', module: 'slf4j-log4j12'
}

testArtifacts
}

ext {
Expand All @@ -95,6 +97,15 @@ subprojects {
testCompile 'org.mockito:mockito-core'
}

task testJar(type: Jar){
archiveClassifier = 'tests'
from sourceSets.test.output
}

artifacts {
testArtifacts testJar
}

publishing {
publications {
nebula(MavenPublication) {
Expand Down Expand Up @@ -284,19 +295,6 @@ project(':iceberg-hive') {
exclude group: 'org.slf4j', module: 'slf4j-log4j12'
}
}

task testJar(type: Jar){
archiveClassifier = 'tests'
from sourceSets.test.output
}

configurations {
testArtifacts
}

artifacts {
testArtifacts testJar
}
}

project(':iceberg-orc') {
Expand Down Expand Up @@ -335,6 +333,8 @@ project(':iceberg-parquet') {
compileOnly("org.apache.hadoop:hadoop-client") {
exclude group: 'org.apache.avro', module: 'avro'
}

testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
* under the License.
*/

package org.apache.iceberg.parquet;
package org.apache.iceberg;

import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
Expand All @@ -32,9 +32,8 @@
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericData.Record;
import org.apache.avro.generic.GenericFixed;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.Schema;
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types.BinaryType;
import org.apache.iceberg.types.Types.BooleanType;
Expand All @@ -55,17 +54,24 @@
import org.junit.Assert;
import org.junit.Test;

import static org.apache.iceberg.Files.localInput;
import static org.apache.iceberg.types.Conversions.fromByteBuffer;
import static org.apache.iceberg.types.Types.NestedField.optional;
import static org.apache.iceberg.types.Types.NestedField.required;

public class TestParquetUtil extends BaseParquetWritingTest {
/**
* Tests for Metrics.
*/
public abstract class TestMetrics {

private final UUID uuid = UUID.randomUUID();
private final GenericFixed fixed = new GenericData.Fixed(
org.apache.avro.Schema.createFixed("fixedCol", null, null, 4),
"abcd".getBytes(StandardCharsets.UTF_8));

public abstract Metrics getMetrics(InputFile file);

public abstract File writeRecords(Schema schema, Record... records) throws IOException;

@Test
public void testMetricsForTopLevelFields() throws IOException {
Schema schema = new Schema(
Expand Down Expand Up @@ -113,9 +119,9 @@ public void testMetricsForTopLevelFields() throws IOException {
secondRecord.put("fixedCol", fixed);
secondRecord.put("binaryCol", "W".getBytes());

File parquetFile = writeRecords(schema, firstRecord, secondRecord);
File recordsFile = writeRecords(schema, firstRecord, secondRecord);

Metrics metrics = ParquetUtil.fileMetrics(localInput(parquetFile));
Metrics metrics = getMetrics(Files.localInput(recordsFile));
Assert.assertEquals(2L, (long) metrics.recordCount());
assertCounts(1, 2L, 0L, metrics);
assertBounds(1, BooleanType.get(), false, true, metrics);
Expand Down Expand Up @@ -160,9 +166,9 @@ public void testMetricsForDecimals() throws IOException {
record.put("decimalAsInt64", new BigDecimal("4.75"));
record.put("decimalAsFixed", new BigDecimal("5.80"));

File parquetFile = writeRecords(schema, record);
File recordsFile = writeRecords(schema, record);

Metrics metrics = ParquetUtil.fileMetrics(localInput(parquetFile));
Metrics metrics = getMetrics(Files.localInput(recordsFile));
Assert.assertEquals(1L, (long) metrics.recordCount());
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, DecimalType.of(4, 2), new BigDecimal("2.55"), new BigDecimal("2.55"), metrics);
Expand Down Expand Up @@ -197,9 +203,9 @@ public void testMetricsForNestedStructFields() throws IOException {
record.put("intCol", Integer.MAX_VALUE);
record.put("nestedStructCol", nestedStruct);

File parquetFile = writeRecords(schema, record);
File recordsFile = writeRecords(schema, record);

Metrics metrics = ParquetUtil.fileMetrics(localInput(parquetFile));
Metrics metrics = getMetrics(Files.localInput(recordsFile));
Assert.assertEquals(1L, (long) metrics.recordCount());
assertCounts(1, 1L, 0L, metrics);
assertBounds(1, IntegerType.get(), Integer.MAX_VALUE, Integer.MAX_VALUE, metrics);
Expand Down Expand Up @@ -232,9 +238,9 @@ public void testMetricsForListAndMapElements() throws IOException {
map.put("4", struct);
record.put(1, map);

File parquetFile = writeRecords(schema, record);
File recordsFile = writeRecords(schema, record);

Metrics metrics = ParquetUtil.fileMetrics(localInput(parquetFile));
Metrics metrics = getMetrics(Files.localInput(recordsFile));
Assert.assertEquals(1L, (long) metrics.recordCount());
assertCounts(1, 1, 0, metrics);
assertBounds(1, IntegerType.get(), null, null, metrics);
Expand All @@ -256,9 +262,9 @@ public void testMetricsForNullColumns() throws IOException {
Record secondRecord = new Record(AvroSchemaUtil.convert(schema.asStruct()));
secondRecord.put("intCol", null);

File parquetFile = writeRecords(schema, firstRecord, secondRecord);
File recordsFile = writeRecords(schema, firstRecord, secondRecord);

Metrics metrics = ParquetUtil.fileMetrics(localInput(parquetFile));
Metrics metrics = getMetrics(Files.localInput(recordsFile));
Assert.assertEquals(2L, (long) metrics.recordCount());
assertCounts(1, 2, 2, metrics);
assertBounds(1, IntegerType.get(), null, null, metrics);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,23 +17,23 @@
* under the License.
*/

package org.apache.iceberg.parquet;
package org.apache.iceberg;

import java.nio.ByteBuffer;
import java.util.Comparator;
import org.apache.iceberg.expressions.Literal;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Comparator;

import static org.apache.iceberg.util.BinaryUtil.truncateBinaryMax;
import static org.apache.iceberg.util.BinaryUtil.truncateBinaryMin;
import static org.apache.iceberg.util.UnicodeUtil.truncateStringMax;
import static org.apache.iceberg.util.UnicodeUtil.truncateStringMin;

public class TestParquetMetricsTruncation {
@SuppressWarnings("checkstyle:LocalVariableName")
public class TestMetricsTruncation {
@Test
public void testTruncateBinaryMin() throws IOException {
public void testTruncateBinaryMin() {
ByteBuffer test1 = ByteBuffer.wrap(new byte[] {1, 1, (byte) 0xFF, 2});
// Output of test1 when truncated to 2 bytes
ByteBuffer test1_2_expected = ByteBuffer.wrap(new byte[] {1, 1});
Expand All @@ -55,7 +55,7 @@ public void testTruncateBinaryMin() throws IOException {
}

@Test
public void testTruncateBinaryMax() throws IOException {
public void testTruncateBinaryMax() {
ByteBuffer test1 = ByteBuffer.wrap(new byte[] {1, 1, 2});
ByteBuffer test2 = ByteBuffer.wrap(new byte[] {1, 1, (byte) 0xFF, 2});
ByteBuffer test3 = ByteBuffer.wrap(new byte[] {(byte) 0xFF, (byte) 0xFF, (byte) 0xFF, 2});
Expand Down Expand Up @@ -83,8 +83,9 @@ public void testTruncateBinaryMax() throws IOException {
cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), expectedOutput) == 0);
}

@SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters")
@Test
public void testTruncateStringMin() throws IOException {
public void testTruncateStringMin() {
String test1 = "イロハニホヘト";
// Output of test1 when truncated to 2 unicode characters
String test1_2_expected = "イロ";
Expand All @@ -96,7 +97,6 @@ public void testTruncateStringMin() throws IOException {
// test4 consists of 2 4 byte UTF-8 characters
String test4 = "\uD800\uDC00\uD800\uDC00";
String test4_1_expected = "\uD800\uDC00";

Comparator<CharSequence> cmp = Literal.of(test1).comparator();
Assert.assertTrue("Truncated lower bound should be lower than or equal to the actual lower bound",
cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1) <= 0);
Expand All @@ -120,8 +120,9 @@ public void testTruncateStringMin() throws IOException {
cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4_1_expected) == 0);
}

@SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters")
@Test
public void testTruncateStringMax() throws IOException {
public void testTruncateStringMax() {
String test1 = "イロハニホヘト";
// Output of test1 when truncated to 2 unicode characters
String test1_2_expected = "イヮ";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,24 +31,23 @@
import org.apache.iceberg.Schema;
import org.apache.iceberg.io.FileAppender;
import org.apache.parquet.schema.MessageType;
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;

import static org.apache.iceberg.Files.localOutput;

/**
* Base utility test class for tests that need to write Parquet files
* Utilities for tests that need to write Parquet files.
*/
public abstract class BaseParquetWritingTest {
class ParquetWritingTestUtils {

@Rule
public TemporaryFolder temp = new TemporaryFolder();
private ParquetWritingTestUtils() {}

File writeRecords(Schema schema, GenericData.Record... records) throws IOException {
return writeRecords(schema, Collections.emptyMap(), null, records);
static File writeRecords(TemporaryFolder temp, Schema schema, GenericData.Record... records) throws IOException {
return writeRecords(temp, schema, Collections.emptyMap(), null, records);
}

File writeRecords(
static File writeRecords(
TemporaryFolder temp,
Schema schema, Map<String, String> properties,
Function<MessageType, ParquetValueWriter<?>> createWriterFunc,
GenericData.Record... records) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,13 +32,19 @@
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.schema.MessageType;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import static org.apache.iceberg.Files.localInput;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
import static org.apache.iceberg.parquet.ParquetWritingTestUtils.writeRecords;
import static org.apache.iceberg.types.Types.NestedField.optional;

public class TestParquet extends BaseParquetWritingTest {
public class TestParquet {

@Rule
public TemporaryFolder temp = new TemporaryFolder();

@Test
public void testRowGroupSizeConfigurable() throws IOException {
Expand Down Expand Up @@ -79,7 +85,7 @@ private File generateFileWithTwoRowGroups(Function<MessageType, ParquetValueWrit
// Force multiple row groups by making the byte size very small
// Note there'a also minimumRowGroupRecordCount which cannot be configured so we have to write
// at least that many records for a new row group to occur
return writeRecords(
return writeRecords(temp,
schema,
ImmutableMap.of(
PARQUET_ROW_GROUP_SIZE_BYTES,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.parquet;

import java.io.File;
import java.io.IOException;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.Metrics;
import org.apache.iceberg.Schema;
import org.apache.iceberg.TestMetrics;
import org.apache.iceberg.io.InputFile;
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;

/**
* Test Metrics for Parquet.
*/
public class TestParquetMetrics extends TestMetrics {

@Rule
public TemporaryFolder temp = new TemporaryFolder();

@Override
public Metrics getMetrics(InputFile file) {
return ParquetUtil.fileMetrics(file);
}

@Override
public File writeRecords(Schema schema, GenericData.Record... records) throws IOException {
return ParquetWritingTestUtils.writeRecords(temp, schema, records);
}
}