|
| 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.iceberg.spark.source; |
| 21 | + |
| 22 | +import java.io.File; |
| 23 | +import java.io.IOException; |
| 24 | +import java.time.LocalDateTime; |
| 25 | +import java.util.List; |
| 26 | +import java.util.Locale; |
| 27 | +import java.util.UUID; |
| 28 | +import java.util.stream.Collectors; |
| 29 | +import org.apache.hadoop.conf.Configuration; |
| 30 | +import org.apache.iceberg.DataFile; |
| 31 | +import org.apache.iceberg.DataFiles; |
| 32 | +import org.apache.iceberg.FileFormat; |
| 33 | +import org.apache.iceberg.PartitionSpec; |
| 34 | +import org.apache.iceberg.Schema; |
| 35 | +import org.apache.iceberg.Table; |
| 36 | +import org.apache.iceberg.data.GenericAppenderFactory; |
| 37 | +import org.apache.iceberg.data.GenericRecord; |
| 38 | +import org.apache.iceberg.data.Record; |
| 39 | +import org.apache.iceberg.hadoop.HadoopTables; |
| 40 | +import org.apache.iceberg.io.FileAppender; |
| 41 | +import org.apache.iceberg.relocated.com.google.common.collect.Lists; |
| 42 | +import org.apache.iceberg.spark.data.GenericsHelpers; |
| 43 | +import org.apache.iceberg.types.Types; |
| 44 | +import org.apache.spark.sql.Dataset; |
| 45 | +import org.apache.spark.sql.Row; |
| 46 | +import org.apache.spark.sql.SparkSession; |
| 47 | +import org.junit.AfterClass; |
| 48 | +import org.junit.Assert; |
| 49 | +import org.junit.Before; |
| 50 | +import org.junit.BeforeClass; |
| 51 | +import org.junit.Rule; |
| 52 | +import org.junit.Test; |
| 53 | +import org.junit.rules.ExpectedException; |
| 54 | +import org.junit.rules.TemporaryFolder; |
| 55 | +import org.junit.runner.RunWith; |
| 56 | +import org.junit.runners.Parameterized; |
| 57 | + |
| 58 | +import static org.apache.iceberg.Files.localOutput; |
| 59 | + |
| 60 | +@RunWith(Parameterized.class) |
| 61 | +public abstract class TestTimestampWithoutZone { |
| 62 | + private static final Configuration CONF = new Configuration(); |
| 63 | + private static final HadoopTables TABLES = new HadoopTables(CONF); |
| 64 | + |
| 65 | + private static final Schema SCHEMA = new Schema( |
| 66 | + Types.NestedField.required(1, "id", Types.LongType.get()), |
| 67 | + Types.NestedField.optional(2, "ts", Types.TimestampType.withoutZone()), |
| 68 | + Types.NestedField.optional(3, "data", Types.StringType.get()) |
| 69 | + ); |
| 70 | + |
| 71 | + private static SparkSession spark = null; |
| 72 | + |
| 73 | + @BeforeClass |
| 74 | + public static void startSpark() { |
| 75 | + TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); |
| 76 | + } |
| 77 | + |
| 78 | + @AfterClass |
| 79 | + public static void stopSpark() { |
| 80 | + SparkSession currentSpark = TestTimestampWithoutZone.spark; |
| 81 | + TestTimestampWithoutZone.spark = null; |
| 82 | + currentSpark.stop(); |
| 83 | + } |
| 84 | + |
| 85 | + @Rule |
| 86 | + public TemporaryFolder temp = new TemporaryFolder(); |
| 87 | + |
| 88 | + private final String format; |
| 89 | + private final boolean vectorized; |
| 90 | + |
| 91 | + @Parameterized.Parameters(name = "format = {0}, vectorized = {1}") |
| 92 | + public static Object[][] parameters() { |
| 93 | + return new Object[][] { |
| 94 | + { "parquet", false }, |
| 95 | + { "parquet", true }, |
| 96 | + { "avro", false }, |
| 97 | + { "orc", false }, |
| 98 | + { "orc", true } |
| 99 | + }; |
| 100 | + } |
| 101 | + |
| 102 | + public TestTimestampWithoutZone(String format, boolean vectorized) { |
| 103 | + this.format = format; |
| 104 | + this.vectorized = vectorized; |
| 105 | + } |
| 106 | + |
| 107 | + private File parent = null; |
| 108 | + private File unpartitioned = null; |
| 109 | + private List<Record> records = null; |
| 110 | + |
| 111 | + @Before |
| 112 | + public void writeUnpartitionedTable() throws IOException { |
| 113 | + this.parent = temp.newFolder("TestTimestampWithoutZone"); |
| 114 | + this.unpartitioned = new File(parent, "unpartitioned"); |
| 115 | + File dataFolder = new File(unpartitioned, "data"); |
| 116 | + Assert.assertTrue("Mkdir should succeed", dataFolder.mkdirs()); |
| 117 | + |
| 118 | + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), unpartitioned.toString()); |
| 119 | + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned |
| 120 | + |
| 121 | + FileFormat fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); |
| 122 | + |
| 123 | + File testFile = new File(dataFolder, fileFormat.addExtension(UUID.randomUUID().toString())); |
| 124 | + |
| 125 | + // create records using the table's schema |
| 126 | + this.records = testRecords(tableSchema); |
| 127 | + |
| 128 | + try (FileAppender<Record> writer = new GenericAppenderFactory(tableSchema).newAppender( |
| 129 | + localOutput(testFile), fileFormat)) { |
| 130 | + writer.addAll(records); |
| 131 | + } |
| 132 | + |
| 133 | + DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) |
| 134 | + .withRecordCount(records.size()) |
| 135 | + .withFileSizeInBytes(testFile.length()) |
| 136 | + .withPath(testFile.toString()) |
| 137 | + .build(); |
| 138 | + |
| 139 | + table.newAppend().appendFile(file).commit(); |
| 140 | + } |
| 141 | + |
| 142 | + @Test |
| 143 | + public void testUnpartitionedTimestampWithoutZone() { |
| 144 | + assertEqualsSafe(SCHEMA.asStruct(), records, read(unpartitioned.toString(), vectorized)); |
| 145 | + } |
| 146 | + |
| 147 | + @Test |
| 148 | + public void testUnpartitionedTimestampWithoutZoneProjection() { |
| 149 | + Schema projection = SCHEMA.select("id", "ts"); |
| 150 | + assertEqualsSafe(projection.asStruct(), |
| 151 | + records.stream().map(r -> projectFlat(projection, r)).collect(Collectors.toList()), |
| 152 | + read(unpartitioned.toString(), vectorized, "id", "ts")); |
| 153 | + } |
| 154 | + |
| 155 | + @Rule |
| 156 | + public ExpectedException exception = ExpectedException.none(); |
| 157 | + |
| 158 | + @Test |
| 159 | + public void testUnpartitionedTimestampWithoutZoneError() { |
| 160 | + exception.expect(IllegalArgumentException.class); |
| 161 | + exception.expectMessage("Spark does not support timestamp without time zone fields"); |
| 162 | + |
| 163 | + spark.read().format("iceberg") |
| 164 | + .option("vectorization-enabled", String.valueOf(vectorized)) |
| 165 | + .option("read-timestamp-without-zone", "false") |
| 166 | + .load(unpartitioned.toString()) |
| 167 | + .collectAsList(); |
| 168 | + } |
| 169 | + |
| 170 | + private static Record projectFlat(Schema projection, Record record) { |
| 171 | + Record result = GenericRecord.create(projection); |
| 172 | + List<Types.NestedField> fields = projection.asStruct().fields(); |
| 173 | + for (int i = 0; i < fields.size(); i += 1) { |
| 174 | + Types.NestedField field = fields.get(i); |
| 175 | + result.set(i, record.getField(field.name())); |
| 176 | + } |
| 177 | + return result; |
| 178 | + } |
| 179 | + |
| 180 | + public static void assertEqualsSafe(Types.StructType struct, |
| 181 | + List<Record> expected, List<Row> actual) { |
| 182 | + Assert.assertEquals("Number of results should match expected", expected.size(), actual.size()); |
| 183 | + for (int i = 0; i < expected.size(); i += 1) { |
| 184 | + GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); |
| 185 | + } |
| 186 | + } |
| 187 | + |
| 188 | + private List<Record> testRecords(Schema schema) { |
| 189 | + return Lists.newArrayList( |
| 190 | + record(schema, 0L, parseToLocal("2017-12-22T09:20:44.294658"), "junction"), |
| 191 | + record(schema, 1L, parseToLocal("2017-12-22T07:15:34.582910"), "alligator"), |
| 192 | + record(schema, 2L, parseToLocal("2017-12-22T06:02:09.243857"), "forrest"), |
| 193 | + record(schema, 3L, parseToLocal("2017-12-22T03:10:11.134509"), "clapping"), |
| 194 | + record(schema, 4L, parseToLocal("2017-12-22T00:34:00.184671"), "brush"), |
| 195 | + record(schema, 5L, parseToLocal("2017-12-21T22:20:08.935889"), "trap"), |
| 196 | + record(schema, 6L, parseToLocal("2017-12-21T21:55:30.589712"), "element"), |
| 197 | + record(schema, 7L, parseToLocal("2017-12-21T17:31:14.532797"), "limited"), |
| 198 | + record(schema, 8L, parseToLocal("2017-12-21T15:21:51.237521"), "global"), |
| 199 | + record(schema, 9L, parseToLocal("2017-12-21T15:02:15.230570"), "goldfish") |
| 200 | + ); |
| 201 | + } |
| 202 | + |
| 203 | + private static List<Row> read(String table, boolean vectorized) { |
| 204 | + return read(table, vectorized, "*"); |
| 205 | + } |
| 206 | + |
| 207 | + private static List<Row> read(String table, boolean vectorized, String select0, String... selectN) { |
| 208 | + Dataset<Row> dataset = spark.read().format("iceberg") |
| 209 | + .option("vectorization-enabled", String.valueOf(vectorized)) |
| 210 | + .option("read-timestamp-without-zone", "true") |
| 211 | + .load(table) |
| 212 | + .select(select0, selectN); |
| 213 | + return dataset.collectAsList(); |
| 214 | + } |
| 215 | + |
| 216 | + private static LocalDateTime parseToLocal(String timestamp) { |
| 217 | + return LocalDateTime.parse(timestamp); |
| 218 | + } |
| 219 | + |
| 220 | + private static Record record(Schema schema, Object... values) { |
| 221 | + Record rec = GenericRecord.create(schema); |
| 222 | + for (int i = 0; i < values.length; i += 1) { |
| 223 | + rec.set(i, values[i]); |
| 224 | + } |
| 225 | + return rec; |
| 226 | + } |
| 227 | +} |
0 commit comments