Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -202,11 +202,6 @@ public Type primitive(Type.PrimitiveType primitive) {
"Cannot project decimal with incompatible precision: %s < %s",
requestedDecimal.precision(), decimal.precision());
break;
case TIMESTAMP:
Types.TimestampType timestamp = (Types.TimestampType) primitive;
Preconditions.checkArgument(timestamp.shouldAdjustToUTC(),
"Cannot project timestamp (without time zone) as timestamptz (with time zone)");
break;
default:
}

Expand Down
25 changes: 25 additions & 0 deletions spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,16 +25,30 @@
import java.util.stream.Collectors;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.hadoop.HadoopFileIO;
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.transforms.Transform;
import org.apache.iceberg.transforms.UnknownTransform;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.Pair;
import org.apache.spark.util.SerializableConfiguration;

public class SparkUtil {

public static final String HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG = "spark-handle-timestamp-without-timezone";
public static final String TIMESTAMP_WITHOUT_TIMEZONE_ERROR = String.format("Cannot handle timestamp without" +
" timezone fields in Spark. Spark does not natively support this type but if you would like to handle all" +
" timestamps as timestamp with timezone set '%s' to true. This will not change the underlying values stored" +
" but will change their displayed values in Spark. For more information please see" +
" https://docs.databricks.com/spark/latest/dataframes-datasets/dates-timestamps.html#ansi-sql-and" +
"-spark-sql-timestamps",
HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG);

private SparkUtil() {
}

Expand Down Expand Up @@ -100,4 +114,15 @@ public static <C, T> Pair<C, T> catalogAndIdentifier(List<String> nameParts,
}
}
}

/**
* Responsible for checking if the table schema has a timestamp without timezone column
* @param schema table schema to check if it contains a timestamp without timezone column
* @return boolean indicating if the schema passed in has a timestamp field without a timezone
*/
public static boolean hasTimestampWithoutZone(Schema schema) {
return TypeUtil.find(schema, t ->
t.typeId().equals(Type.TypeID.TIMESTAMP) && !((Types.TimestampType) t).shouldAdjustToUTC()
) != null;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -104,12 +104,14 @@ public DataType primitive(Type.PrimitiveType primitive) {
throw new UnsupportedOperationException(
"Spark does not support time fields");
case TIMESTAMP:
Types.TimestampType timestamp = (Types.TimestampType) primitive;
if (timestamp.shouldAdjustToUTC()) {
return TimestampType$.MODULE$;
}
throw new UnsupportedOperationException(
"Spark does not support timestamp without time zone fields");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we do the flag check here as well to check whether or not we have enabled the "Handle without timezone" flag here as well? We may be using this not on the read path (like in the migrate/snapshot code) and it would be good to catch it here as well and make sure users know what is happening.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think this might involve a bigger refactor, including changing the method signature to accept the flag to .primitive(..., handleTimestampWithoutTimezoneFlag). I'm not sure if that will break other stuff. I added in the logic as comments for now as to how to implement it after we can settle on the implementation

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that's fair, I can always fix this in the migrate code directly

// Types.TimestampType timestamp = (Types.TimestampType) primitive;
// boolean readTimestampWithoutZone = options.get(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAGz)
// .map(Boolean::parseBoolean).orElse(false);
// if (timestamp.shouldAdjustToUTC() || readTimestampWithoutZone) {
// return TimestampType$.MODULE$;
// }
// throw new UnsupportedOperationException(SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
return TimestampType$.MODULE$;
case STRING:
return StringType$.MODULE$;
case UUID:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import java.sql.Timestamp;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.time.temporal.ChronoUnit;
Expand Down Expand Up @@ -122,13 +123,19 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual)
Assert.assertEquals("ISO-8601 date should be equal", expected.toString(), actual.toString());
break;
case TIMESTAMP:
Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime);
Assert.assertTrue("Should be a Timestamp", actual instanceof Timestamp);
Timestamp ts = (Timestamp) actual;
// milliseconds from nanos has already been added by getTime
OffsetDateTime actualTs = EPOCH.plusNanos(
(ts.getTime() * 1_000_000) + (ts.getNanos() % 1_000_000));
Assert.assertEquals("Timestamp should be equal", expected, actualTs);
Types.TimestampType timestampType = (Types.TimestampType) type;
if (timestampType.shouldAdjustToUTC()) {
Assert.assertTrue("Should expect an OffsetDateTime", expected instanceof OffsetDateTime);
Assert.assertEquals("Timestamp should be equal", expected, actualTs);
} else {
Assert.assertTrue("Should expect an LocalDateTime", expected instanceof LocalDateTime);
Assert.assertEquals("Timestamp should be equal", expected, actualTs.toLocalDateTime());
}
break;
case STRING:
Assert.assertTrue("Should be a String", actual instanceof String);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,227 @@
/*
* 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.spark.source;

import java.io.File;
import java.io.IOException;
import java.time.LocalDateTime;
import java.util.List;
import java.util.Locale;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.AssertHelpers;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.DataFiles;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.data.GenericAppenderFactory;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.spark.SparkUtil;
import org.apache.iceberg.spark.data.GenericsHelpers;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;

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

@RunWith(Parameterized.class)
public abstract class TestTimestampWithoutZone {
private static final Configuration CONF = new Configuration();
private static final HadoopTables TABLES = new HadoopTables(CONF);

private static final Schema SCHEMA = new Schema(
Types.NestedField.required(1, "id", Types.LongType.get()),
Types.NestedField.optional(2, "ts", Types.TimestampType.withoutZone()),
Types.NestedField.optional(3, "data", Types.StringType.get())
);

private static SparkSession spark = null;

@BeforeClass
public static void startSpark() {
TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate();
}

@AfterClass
public static void stopSpark() {
SparkSession currentSpark = TestTimestampWithoutZone.spark;
TestTimestampWithoutZone.spark = null;
currentSpark.stop();
}

@Rule
public TemporaryFolder temp = new TemporaryFolder();

private final String format;
private final boolean vectorized;

@Parameterized.Parameters(name = "format = {0}, vectorized = {1}")
public static Object[][] parameters() {
return new Object[][] {
{ "parquet", false },
{ "parquet", true },
{ "avro", false }
};
}

public TestTimestampWithoutZone(String format, boolean vectorized) {
this.format = format;
this.vectorized = vectorized;
}

private File parent = null;
private File unpartitioned = null;
private List<Record> records = null;

@Before
public void writeUnpartitionedTable() throws IOException {
this.parent = temp.newFolder("TestTimestampWithoutZone");
this.unpartitioned = new File(parent, "unpartitioned");
File dataFolder = new File(unpartitioned, "data");
Assert.assertTrue("Mkdir should succeed", dataFolder.mkdirs());

Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), unpartitioned.toString());
Schema tableSchema = table.schema(); // use the table schema because ids are reassigned

FileFormat fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH));

File testFile = new File(dataFolder, fileFormat.addExtension(UUID.randomUUID().toString()));

// create records using the table's schema
this.records = testRecords(tableSchema);

try (FileAppender<Record> writer = new GenericAppenderFactory(tableSchema).newAppender(
localOutput(testFile), fileFormat)) {
writer.addAll(records);
}

DataFile file = DataFiles.builder(PartitionSpec.unpartitioned())
.withRecordCount(records.size())
.withFileSizeInBytes(testFile.length())
.withPath(testFile.toString())
.build();

table.newAppend().appendFile(file).commit();
}

@Test
public void testUnpartitionedTimestampWithoutZone() {
assertEqualsSafe(SCHEMA.asStruct(), records, read(unpartitioned.toString(), vectorized));
}

@Test
public void testUnpartitionedTimestampWithoutZoneProjection() {
Schema projection = SCHEMA.select("id", "ts");
assertEqualsSafe(projection.asStruct(),
records.stream().map(r -> projectFlat(projection, r)).collect(Collectors.toList()),
read(unpartitioned.toString(), vectorized, "id", "ts"));
}

@Rule
public ExpectedException exception = ExpectedException.none();

@Test
public void testUnpartitionedTimestampWithoutZoneError() {
AssertHelpers.assertThrows(String.format("Read operation performed on a timestamp without timezone field while " +
"'%s' set to false should throw exception", SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG),
IllegalArgumentException.class, SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, () -> spark.read()
.format("iceberg")
.option("vectorization-enabled", String.valueOf(vectorized))
.option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG, "false")
.load(unpartitioned.toString())
.collectAsList());
}

private static Record projectFlat(Schema projection, Record record) {
Record result = GenericRecord.create(projection);
List<Types.NestedField> fields = projection.asStruct().fields();
for (int i = 0; i < fields.size(); i += 1) {
Types.NestedField field = fields.get(i);
result.set(i, record.getField(field.name()));
}
return result;
}

public static void assertEqualsSafe(Types.StructType struct,
List<Record> expected, List<Row> actual) {
Assert.assertEquals("Number of results should match expected", expected.size(), actual.size());
for (int i = 0; i < expected.size(); i += 1) {
GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i));
}
}

private List<Record> testRecords(Schema schema) {
return Lists.newArrayList(
record(schema, 0L, parseToLocal("2017-12-22T09:20:44.294658"), "junction"),
record(schema, 1L, parseToLocal("2017-12-22T07:15:34.582910"), "alligator"),
record(schema, 2L, parseToLocal("2017-12-22T06:02:09.243857"), "forrest"),
record(schema, 3L, parseToLocal("2017-12-22T03:10:11.134509"), "clapping"),
record(schema, 4L, parseToLocal("2017-12-22T00:34:00.184671"), "brush"),
record(schema, 5L, parseToLocal("2017-12-21T22:20:08.935889"), "trap"),
record(schema, 6L, parseToLocal("2017-12-21T21:55:30.589712"), "element"),
record(schema, 7L, parseToLocal("2017-12-21T17:31:14.532797"), "limited"),
record(schema, 8L, parseToLocal("2017-12-21T15:21:51.237521"), "global"),
record(schema, 9L, parseToLocal("2017-12-21T15:02:15.230570"), "goldfish")
);
}

private static List<Row> read(String table, boolean vectorized) {
return read(table, vectorized, "*");
}

private static List<Row> read(String table, boolean vectorized, String select0, String... selectN) {
Dataset<Row> dataset = spark.read().format("iceberg")
.option("vectorization-enabled", String.valueOf(vectorized))
.option(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG, "true")
.load(table)
.select(select0, selectN);
return dataset.collectAsList();
}

private static LocalDateTime parseToLocal(String timestamp) {
return LocalDateTime.parse(timestamp);
}

private static Record record(Schema schema, Object... values) {
Record rec = GenericRecord.create(schema);
for (int i = 0; i < values.length; i += 1) {
rec.set(i, values[i]);
}
return rec;
}
}
16 changes: 15 additions & 1 deletion spark2/src/main/java/org/apache/iceberg/spark/source/Reader.java
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@
import org.apache.iceberg.spark.SparkFilters;
import org.apache.iceberg.spark.SparkReadOptions;
import org.apache.iceberg.spark.SparkSchemaUtil;
import org.apache.iceberg.spark.SparkUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.TableScanUtil;
import org.apache.spark.broadcast.Broadcast;
Expand Down Expand Up @@ -98,6 +99,7 @@ class Reader implements DataSourceReader, SupportsScanColumnarBatch, SupportsPus
private Filter[] pushedFilters = NO_FILTERS;
private final boolean localityPreferred;
private final int batchSize;
private final boolean readTimestampWithoutZone;

// lazy variables
private Schema schema = null;
Expand Down Expand Up @@ -162,6 +164,14 @@ class Reader implements DataSourceReader, SupportsScanColumnarBatch, SupportsPus
this.batchSize = options.get(SparkReadOptions.VECTORIZATION_BATCH_SIZE).map(Integer::parseInt).orElseGet(() ->
PropertyUtil.propertyAsInt(table.properties(),
TableProperties.PARQUET_BATCH_SIZE, TableProperties.PARQUET_BATCH_SIZE_DEFAULT));
// Allow reading timestamp without time zone as timestamp with time zone. Generally, this is not safe as timestamp
// without time zone is supposed to represent wall clock time semantics, i.e. no matter the reader/writer timezone
// 3PM should always be read as 3PM, but timestamp with time zone represents instant semantics, i.e the timestamp
// is adjusted so that the corresponding time in the reader timezone is displayed.
// When set to false (default), we throw an exception at runtime
// "Spark does not support timestamp without time zone fields" if reading timestamp without time zone fields
this.readTimestampWithoutZone = options.get(SparkUtil.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE_FLAG)
.map(Boolean::parseBoolean).orElse(false);
}

private Schema lazySchema() {
Expand All @@ -185,6 +195,8 @@ private Expression filterExpression() {

private StructType lazyType() {
if (type == null) {
Preconditions.checkArgument(readTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(lazySchema()),
SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
this.type = SparkSchemaUtil.convert(lazySchema());
}
return type;
Expand Down Expand Up @@ -332,8 +344,10 @@ public boolean enableBatchRead() {

boolean batchReadsEnabled = batchReadsEnabled(allParquetFileScanTasks, allOrcFileScanTasks);

boolean hasTimestampWithoutZone = SparkUtil.hasTimestampWithoutZone(lazySchema());

this.readUsingBatch = batchReadsEnabled && hasNoDeleteFiles && (allOrcFileScanTasks ||
(allParquetFileScanTasks && atLeastOneColumn && onlyPrimitives));
(allParquetFileScanTasks && atLeastOneColumn && onlyPrimitives && !hasTimestampWithoutZone));
}
return readUsingBatch;
}
Expand Down
Loading