|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.apache.parquet.tools.read; |
| 21 | + |
| 22 | +import org.apache.hadoop.conf.Configuration; |
| 23 | +import org.apache.hadoop.fs.Path; |
| 24 | +import org.apache.parquet.column.ParquetProperties; |
| 25 | +import org.apache.parquet.example.data.Group; |
| 26 | +import org.apache.parquet.example.data.simple.SimpleGroupFactory; |
| 27 | +import org.apache.parquet.hadoop.ParquetReader; |
| 28 | +import org.apache.parquet.hadoop.ParquetWriter; |
| 29 | +import org.apache.parquet.hadoop.example.GroupWriteSupport; |
| 30 | +import org.apache.parquet.hadoop.metadata.CompressionCodecName; |
| 31 | +import org.apache.parquet.io.api.Binary; |
| 32 | +import org.apache.parquet.schema.MessageType; |
| 33 | +import org.apache.parquet.schema.PrimitiveType; |
| 34 | +import org.apache.parquet.schema.Type; |
| 35 | +import org.junit.Assert; |
| 36 | +import org.junit.Before; |
| 37 | +import org.junit.Rule; |
| 38 | +import org.junit.Test; |
| 39 | +import org.junit.rules.TemporaryFolder; |
| 40 | + |
| 41 | +import java.io.File; |
| 42 | +import java.io.IOException; |
| 43 | + |
| 44 | +public class TestSimpleRecordConverter { |
| 45 | + |
| 46 | + private static final String INT32_FIELD = "int32_field"; |
| 47 | + private static final String INT64_FIELD = "int64_field"; |
| 48 | + private static final String FLOAT_FIELD = "float_field"; |
| 49 | + private static final String DOUBLE_FIELD = "double_field"; |
| 50 | + private static final String BINARY_FIELD = "binary_field"; |
| 51 | + private static final String FIXED_LEN_BYTE_ARRAY_FIELD = "flba_field"; |
| 52 | + |
| 53 | + @Rule |
| 54 | + public TemporaryFolder tempFolder = new TemporaryFolder(); |
| 55 | + |
| 56 | + @Test |
| 57 | + public void testConverter() throws IOException { |
| 58 | + try ( |
| 59 | + ParquetReader<SimpleRecord> reader = |
| 60 | + ParquetReader.builder(new SimpleReadSupport(), new Path(testFile().getAbsolutePath())).build()) { |
| 61 | + for (SimpleRecord record = reader.read(); record != null; record = reader.read()) { |
| 62 | + for (SimpleRecord.NameValue value : record.getValues()) { |
| 63 | + switch(value.getName()) { |
| 64 | + case INT32_FIELD: |
| 65 | + Assert.assertEquals(32, value.getValue()); |
| 66 | + break; |
| 67 | + case INT64_FIELD: |
| 68 | + Assert.assertEquals(64L, value.getValue()); |
| 69 | + break; |
| 70 | + case FLOAT_FIELD: |
| 71 | + Assert.assertEquals(1.0f, value.getValue()); |
| 72 | + break; |
| 73 | + case DOUBLE_FIELD: |
| 74 | + Assert.assertEquals(2.0d, value.getValue()); |
| 75 | + break; |
| 76 | + case BINARY_FIELD: |
| 77 | + Assert.assertArrayEquals("foobar".getBytes(), (byte[])value.getValue()); |
| 78 | + break; |
| 79 | + case FIXED_LEN_BYTE_ARRAY_FIELD: |
| 80 | + Assert.assertArrayEquals(new byte[]{ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 }, (byte[])value.getValue()); |
| 81 | + break; |
| 82 | + } |
| 83 | + } |
| 84 | + } |
| 85 | + } |
| 86 | + } |
| 87 | + |
| 88 | + @Before |
| 89 | + public void setUp() throws IOException { |
| 90 | + createTestParquetFile(); |
| 91 | + } |
| 92 | + |
| 93 | + private static MessageType createSchema() { |
| 94 | + return new MessageType("schema", |
| 95 | + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT32, INT32_FIELD), |
| 96 | + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.INT64, INT64_FIELD), |
| 97 | + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.FLOAT, FLOAT_FIELD), |
| 98 | + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.DOUBLE, DOUBLE_FIELD), |
| 99 | + new PrimitiveType(Type.Repetition.REQUIRED, PrimitiveType.PrimitiveTypeName.BINARY, BINARY_FIELD), |
| 100 | + new PrimitiveType(Type.Repetition.REQUIRED, |
| 101 | + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, 12, FIXED_LEN_BYTE_ARRAY_FIELD) |
| 102 | + ); |
| 103 | + } |
| 104 | + |
| 105 | + private void createTestParquetFile() throws IOException { |
| 106 | + Path fsPath = new Path(testFile().getPath()); |
| 107 | + Configuration conf = new Configuration(); |
| 108 | + |
| 109 | + MessageType schema = createSchema(); |
| 110 | + SimpleGroupFactory fact = new SimpleGroupFactory(schema); |
| 111 | + GroupWriteSupport.setSchema(schema, conf); |
| 112 | + |
| 113 | + try ( |
| 114 | + ParquetWriter<Group> writer = new ParquetWriter<>( |
| 115 | + fsPath, |
| 116 | + new GroupWriteSupport(), |
| 117 | + CompressionCodecName.UNCOMPRESSED, |
| 118 | + 1024, |
| 119 | + 1024, |
| 120 | + 512, |
| 121 | + true, |
| 122 | + false, |
| 123 | + ParquetProperties.WriterVersion.PARQUET_2_0, |
| 124 | + conf)) { |
| 125 | + writer.write(fact.newGroup() |
| 126 | + .append(INT32_FIELD, 32) |
| 127 | + .append(INT64_FIELD, 64L) |
| 128 | + .append(FLOAT_FIELD, 1.0f) |
| 129 | + .append(DOUBLE_FIELD, 2.0d) |
| 130 | + .append(BINARY_FIELD, Binary.fromString("foobar")) |
| 131 | + .append(FIXED_LEN_BYTE_ARRAY_FIELD, |
| 132 | + Binary.fromConstantByteArray(new byte[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 }))); |
| 133 | + } |
| 134 | + } |
| 135 | + |
| 136 | + private File testFile() { |
| 137 | + return new File(this.tempFolder.getRoot(), getClass().getSimpleName() + ".parquet"); |
| 138 | + } |
| 139 | +} |
0 commit comments