From fb5df209e42669ac5a98082ddcf2cea9e5a3687c Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Wed, 19 Sep 2018 11:47:51 +0200 Subject: [PATCH 01/26] Timezone adjustment --- .../parquet/VectorizedColumnReader.java | 73 ++++- .../VectorizedParquetRecordReader.java | 9 +- .../parquet/ParquetFileFormat.scala | 10 +- .../datasources/parquet/ParquetFilters.scala | 308 ++++++++++-------- .../parquet/ParquetReadSupport.scala | 24 +- .../parquet/ParquetRecordMaterializer.scala | 6 +- .../parquet/ParquetRowConverter.scala | 39 ++- .../parquet/ParquetSchemaConverter.scala | 104 +++--- .../test-data/timestamp_dictionary.parq | Bin 0 -> 1209 bytes .../test-data/timestamp_dictionary.txt | 4 + .../resources/test-data/timestamp_plain.parq | Bin 0 -> 1099 bytes .../resources/test-data/timestamp_plain.txt | 2 + .../benchmark/DataSourceReadBenchmark.scala | 7 +- .../parquet/ParquetEncodingSuite.scala | 11 +- .../datasources/parquet/ParquetIOSuite.scala | 17 +- .../ParquetInteroperabilitySuite.scala | 48 ++- 16 files changed, 423 insertions(+), 239 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/timestamp_dictionary.parq create mode 100644 sql/core/src/test/resources/test-data/timestamp_dictionary.txt create mode 100644 sql/core/src/test/resources/test-data/timestamp_plain.parq create mode 100644 sql/core/src/test/resources/test-data/timestamp_plain.txt diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index ba26b57567e6..60da7cb4e2d9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -30,7 +30,8 @@ import org.apache.parquet.column.page.*; import org.apache.parquet.column.values.ValuesReader; import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.TimestampLogicalTypeAnnotation; import org.apache.parquet.schema.PrimitiveType; import org.apache.spark.sql.catalyst.util.DateTimeUtils; @@ -40,6 +41,8 @@ import org.apache.spark.sql.types.DecimalType; import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MICROS; +import static org.apache.parquet.schema.LogicalTypeAnnotation.TimeUnit.MILLIS; import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.ValuesReaderIntIterator; import static org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.createRLEIterator; @@ -96,20 +99,23 @@ public class VectorizedColumnReader { private final PageReader pageReader; private final ColumnDescriptor descriptor; - private final OriginalType originalType; + private final LogicalTypeAnnotation logicalTypeAnnotation; // The timezone conversion to apply to int96 timestamps. Null if no conversion. private final TimeZone convertTz; + private final TimeZone sessionLocalTz; private static final TimeZone UTC = DateTimeUtils.TimeZoneUTC(); public VectorizedColumnReader( ColumnDescriptor descriptor, - OriginalType originalType, + LogicalTypeAnnotation logicalTypeAnnotation, PageReader pageReader, - TimeZone convertTz) throws IOException { + TimeZone convertTz, + TimeZone sessionLocalTz) throws IOException { this.descriptor = descriptor; this.pageReader = pageReader; this.convertTz = convertTz; - this.originalType = originalType; + this.sessionLocalTz = sessionLocalTz; + this.logicalTypeAnnotation = logicalTypeAnnotation; this.maxDefLevel = descriptor.getMaxDefinitionLevel(); DictionaryPage dictionaryPage = pageReader.readDictionaryPage(); @@ -179,7 +185,7 @@ void readBatch(int total, WritableColumnVector column) throws IOException { if (column.hasDictionary() || (rowId == 0 && (typeName == PrimitiveType.PrimitiveTypeName.INT32 || (typeName == PrimitiveType.PrimitiveTypeName.INT64 && - originalType != OriginalType.TIMESTAMP_MILLIS) || + !(logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation)) || typeName == PrimitiveType.PrimitiveTypeName.FLOAT || typeName == PrimitiveType.PrimitiveTypeName.DOUBLE || typeName == PrimitiveType.PrimitiveTypeName.BINARY))) { @@ -287,17 +293,24 @@ private void decodeDictionaryIds( case INT64: if (column.dataType() == DataTypes.LongType || DecimalType.is64BitDecimalType(column.dataType()) || - originalType == OriginalType.TIMESTAMP_MICROS) { + isTimestampWithUnit(logicalTypeAnnotation, MICROS)) { for (int i = rowId; i < rowId + num; ++i) { if (!column.isNullAt(i)) { - column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i))); + long time = dictionary.decodeToLong(dictionaryIds.getDictId(i)); + if (needsTimezoneAdjustment()) { + time = DateTimeUtils.convertTz(time, sessionLocalTz, UTC); + } + column.putLong(i, time); } } - } else if (originalType == OriginalType.TIMESTAMP_MILLIS) { + } else if (isTimestampWithUnit(logicalTypeAnnotation, MILLIS)) { for (int i = rowId; i < rowId + num; ++i) { if (!column.isNullAt(i)) { - column.putLong(i, - DateTimeUtils.fromMillis(dictionary.decodeToLong(dictionaryIds.getDictId(i)))); + long time = DateTimeUtils.fromMillis(dictionary.decodeToLong(dictionaryIds.getDictId(i))); + if (needsTimezoneAdjustment()) { + time = DateTimeUtils.convertTz(time, sessionLocalTz, UTC); + } + column.putLong(i, time); } } } else { @@ -425,13 +438,31 @@ private void readLongBatch(int rowId, int num, WritableColumnVector column) thro // This is where we implement support for the valid type conversions. if (column.dataType() == DataTypes.LongType || DecimalType.is64BitDecimalType(column.dataType()) || - originalType == OriginalType.TIMESTAMP_MICROS) { - defColumn.readLongs( - num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); - } else if (originalType == OriginalType.TIMESTAMP_MILLIS) { + isTimestampWithUnit(logicalTypeAnnotation, MICROS)) { + if (needsTimezoneAdjustment()) { + for (int i = 0; i < num; i++) { + if (defColumn.readInteger() == maxDefLevel) { + long timestamp = dataColumn.readLong(); + if (needsTimezoneAdjustment()) { + timestamp = DateTimeUtils.convertTz(timestamp, sessionLocalTz, UTC); + } + column.putLong(rowId + i, timestamp); + } else { + column.putNull(rowId + i); + } + } + } else { + defColumn.readLongs( + num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn); + } + } else if (isTimestampWithUnit(logicalTypeAnnotation, MILLIS)) { for (int i = 0; i < num; i++) { if (defColumn.readInteger() == maxDefLevel) { - column.putLong(rowId + i, DateTimeUtils.fromMillis(dataColumn.readLong())); + long timestamp = DateTimeUtils.fromMillis(dataColumn.readLong()); + if (needsTimezoneAdjustment()) { + timestamp = DateTimeUtils.convertTz(timestamp, sessionLocalTz, UTC); + } + column.putLong(rowId + i, timestamp); } else { column.putNull(rowId + i); } @@ -441,6 +472,16 @@ private void readLongBatch(int rowId, int num, WritableColumnVector column) thro } } + private boolean needsTimezoneAdjustment() { + return logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation && + !((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC(); + } + + private boolean isTimestampWithUnit(LogicalTypeAnnotation logicalTypeAnnotation, LogicalTypeAnnotation.TimeUnit timeUnit) { + return (logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation) && + ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == timeUnit; + } + private void readFloatBatch(int rowId, int num, WritableColumnVector column) throws IOException { // This is where we implement support for the valid type conversions. // TODO: support implicit cast to double? diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java index f02861355c40..6c72b7523435 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java @@ -88,6 +88,8 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa */ private TimeZone convertTz = null; + private TimeZone sessionLocalTz = null; + /** * columnBatch object that is used for batch decoding. This is created on first use and triggers * batched decoding. It is not valid to interleave calls to the batched interface with the row @@ -116,8 +118,9 @@ public class VectorizedParquetRecordReader extends SpecificParquetRecordReaderBa */ private final MemoryMode MEMORY_MODE; - public VectorizedParquetRecordReader(TimeZone convertTz, boolean useOffHeap, int capacity) { + public VectorizedParquetRecordReader(TimeZone convertTz, TimeZone sessionLocalTz, boolean useOffHeap, int capacity) { this.convertTz = convertTz; + this.sessionLocalTz = sessionLocalTz; MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP; this.capacity = capacity; } @@ -308,8 +311,8 @@ private void checkEndOfRowGroup() throws IOException { columnReaders = new VectorizedColumnReader[columns.size()]; for (int i = 0; i < columns.size(); ++i) { if (missingColumns[i]) continue; - columnReaders[i] = new VectorizedColumnReader(columns.get(i), types.get(i).getOriginalType(), - pages.getPageReader(columns.get(i)), convertTz); + columnReaders[i] = new VectorizedColumnReader(columns.get(i), types.get(i).getLogicalTypeAnnotation(), + pages.getPageReader(columns.get(i)), convertTz, sessionLocalTz); } totalCountLoadedSoFar += pages.getRowCount(); } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index efa4f3f166d9..c95ef0bcc18f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -395,6 +395,8 @@ class ParquetFileFormat } else { None } + val sessionLocalTz = DateTimeUtils.getTimeZone( + sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = @@ -408,7 +410,8 @@ class ParquetFileFormat val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) + convertTz.orNull, sessionLocalTz, + enableOffHeapColumnVector && taskContext.isDefined, capacity) val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion lister before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) @@ -426,9 +429,10 @@ class ParquetFileFormat // ParquetRecordReader returns UnsafeRow val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz, sessionLocalTz), + parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz, sessionLocalTz)) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 21ab9c78e53d..e59d7e68d0ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -23,12 +23,13 @@ import java.sql.{Date, Timestamp} import java.util.Locale import scala.collection.JavaConverters.asScalaBufferConverter +import scala.language.existentials import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.io.api.Binary -import org.apache.parquet.schema.{DecimalMetadata, MessageType, OriginalType, PrimitiveComparator} -import org.apache.parquet.schema.OriginalType._ +import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ @@ -59,23 +60,25 @@ private[parquet] class ParquetFilters( fieldType: ParquetSchemaType) private case class ParquetSchemaType( - originalType: OriginalType, - primitiveTypeName: PrimitiveTypeName, - length: Int, - decimalMetadata: DecimalMetadata) - - private val ParquetBooleanType = ParquetSchemaType(null, BOOLEAN, 0, null) - private val ParquetByteType = ParquetSchemaType(INT_8, INT32, 0, null) - private val ParquetShortType = ParquetSchemaType(INT_16, INT32, 0, null) - private val ParquetIntegerType = ParquetSchemaType(null, INT32, 0, null) - private val ParquetLongType = ParquetSchemaType(null, INT64, 0, null) - private val ParquetFloatType = ParquetSchemaType(null, FLOAT, 0, null) - private val ParquetDoubleType = ParquetSchemaType(null, DOUBLE, 0, null) - private val ParquetStringType = ParquetSchemaType(UTF8, BINARY, 0, null) - private val ParquetBinaryType = ParquetSchemaType(null, BINARY, 0, null) - private val ParquetDateType = ParquetSchemaType(DATE, INT32, 0, null) - private val ParquetTimestampMicrosType = ParquetSchemaType(TIMESTAMP_MICROS, INT64, 0, null) - private val ParquetTimestampMillisType = ParquetSchemaType(TIMESTAMP_MILLIS, INT64, 0, null) + logicalType: LogicalTypeAnnotation, + logicalTypeClass: Class[_ <: LogicalTypeAnnotation], + primitiveTypeName: PrimitiveTypeName, + length: Int) + + private val ParquetBooleanType = ParquetSchemaType(null, null, BOOLEAN, 0) + private val ParquetByteType = ParquetSchemaType(intType(8, true), + classOf[IntLogicalTypeAnnotation], INT32, 0) + private val ParquetShortType = ParquetSchemaType(intType(16, true), + classOf[IntLogicalTypeAnnotation], INT32, 0) + private val ParquetIntegerType = ParquetSchemaType(null, null, INT32, 0) + private val ParquetLongType = ParquetSchemaType(null, null, INT64, 0) + private val ParquetFloatType = ParquetSchemaType(null, null, FLOAT, 0) + private val ParquetDoubleType = ParquetSchemaType(null, null, DOUBLE, 0) + private val ParquetStringType = ParquetSchemaType(stringType(), + classOf[StringLogicalTypeAnnotation], BINARY, 0) + private val ParquetBinaryType = ParquetSchemaType(null, null, BINARY, 0) + private val ParquetDateType = ParquetSchemaType(dateType(), + classOf[DateLogicalTypeAnnotation], INT32, 0) private def dateToDays(date: Date): SQLDate = { DateTimeUtils.fromJavaDate(date) @@ -104,9 +107,10 @@ private[parquet] class ParquetFilters( case ParquetBooleanType => (n: String, v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[JBoolean]) case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: String, v: Any) => FilterApi.eq( - intColumn(n), - Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) + (n: String, v: Any) => + FilterApi.eq( + intColumn(n), + Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) case ParquetLongType => (n: String, v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[JLong]) case ParquetFloatType => @@ -116,39 +120,48 @@ private[parquet] class ParquetFilters( // Binary.fromString and Binary.fromByteArray don't accept null values case ParquetStringType => - (n: String, v: Any) => FilterApi.eq( - binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + (n: String, v: Any) => + FilterApi.eq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) case ParquetBinaryType => - (n: String, v: Any) => FilterApi.eq( - binaryColumn(n), - Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + (n: String, v: Any) => + FilterApi.eq( + binaryColumn(n), + Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) case ParquetDateType if pushDownDate => - (n: String, v: Any) => FilterApi.eq( - intColumn(n), - Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.eq( - longColumn(n), - Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong]).orNull) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.eq( - longColumn(n), - Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) - - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => - (n: String, v: Any) => FilterApi.eq( - intColumn(n), - Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => - (n: String, v: Any) => FilterApi.eq( - longColumn(n), - Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => - (n: String, v: Any) => FilterApi.eq( - binaryColumn(n), - Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) + (n: String, v: Any) => + FilterApi.eq( + intColumn(n), + Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.eq( + longColumn(n), + Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) + .asInstanceOf[JLong]).orNull) + } else { + (n: String, v: Any) => FilterApi.eq( + longColumn(n), + Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) + } + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => + (n: String, v: Any) => FilterApi.eq( + intColumn(n), + Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), + Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => + (n: String, v: Any) => FilterApi.eq( + binaryColumn(n), + Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) } private val makeNotEq: PartialFunction[ParquetSchemaType, (String, Any) => FilterPredicate] = { @@ -177,25 +190,32 @@ private[parquet] class ParquetFilters( (n: String, v: Any) => FilterApi.notEq( intColumn(n), Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.notEq( - longColumn(n), - Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong]).orNull) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.notEq( - longColumn(n), - Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) - - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => - (n: String, v: Any) => FilterApi.notEq( - intColumn(n), - Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.notEq( + longColumn(n), + Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) + .asInstanceOf[JLong]).orNull) + } else { + (n: String, v: Any) => + FilterApi.notEq( + longColumn(n), + Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) + } + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => + (n: String, v: Any) => FilterApi.notEq( + intColumn(n), + Option(v).map(d => decimalToInt32(d.asInstanceOf[JBigDecimal])).orNull) + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.notEq( longColumn(n), Option(v).map(d => decimalToInt64(d.asInstanceOf[JBigDecimal])).orNull) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), Option(v).map(d => decimalToByteArray(d.asInstanceOf[JBigDecimal], length)).orNull) @@ -221,22 +241,30 @@ private[parquet] class ParquetFilters( case ParquetDateType if pushDownDate => (n: String, v: Any) => FilterApi.lt(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.lt( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.lt( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.lt( + longColumn(n), + DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) + } else { + (n: String, v: Any) => + FilterApi.lt( + longColumn(n), + v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) + } - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.lt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.lt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) } @@ -261,22 +289,29 @@ private[parquet] class ParquetFilters( case ParquetDateType if pushDownDate => (n: String, v: Any) => FilterApi.ltEq(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.ltEq( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.ltEq( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.ltEq( + longColumn(n), + DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) + } else { + (n: String, v: Any) => + FilterApi.ltEq( + longColumn(n), + v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) + } + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.ltEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.ltEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) } @@ -301,22 +336,29 @@ private[parquet] class ParquetFilters( case ParquetDateType if pushDownDate => (n: String, v: Any) => FilterApi.gt(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.gt( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.gt( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.gt( + longColumn(n), + DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) + } else { + (n: String, v: Any) => + FilterApi.gt( + longColumn(n), + v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) + } + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gt(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gt(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) } @@ -341,22 +383,29 @@ private[parquet] class ParquetFilters( case ParquetDateType if pushDownDate => (n: String, v: Any) => FilterApi.gtEq(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) - case ParquetTimestampMicrosType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.gtEq( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - case ParquetTimestampMillisType if pushDownTimestamp => - (n: String, v: Any) => FilterApi.gtEq( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - - case ParquetSchemaType(DECIMAL, INT32, _, _) if pushDownDecimal => + case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && + _class == classOf[TimestampLogicalTypeAnnotation] => + if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { + (n: String, v: Any) => + FilterApi.gtEq( + longColumn(n), + DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) + } else { + (n: String, v: Any) => + FilterApi.gtEq( + longColumn(n), + v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) + } + case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gtEq(intColumn(n), decimalToInt32(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, INT64, _, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, INT64, _) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gtEq(longColumn(n), decimalToInt64(v.asInstanceOf[JBigDecimal])) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, length, _) if pushDownDecimal => + case ParquetSchemaType(_, _class, FIXED_LEN_BYTE_ARRAY, length) if pushDownDecimal && + _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) } @@ -371,8 +420,9 @@ private[parquet] class ParquetFilters( val primitiveFields = dataType.getFields.asScala.filter(_.isPrimitive).map(_.asPrimitiveType()).map { f => f.getName -> ParquetField(f.getName, - ParquetSchemaType(f.getOriginalType, - f.getPrimitiveTypeName, f.getTypeLength, f.getDecimalMetadata)) + ParquetSchemaType(f.getLogicalTypeAnnotation, + if (f.getLogicalTypeAnnotation == null) null else f.getLogicalTypeAnnotation.getClass, + f.getPrimitiveTypeName, f.getTypeLength)) } if (caseSensitive) { primitiveFields.toMap @@ -412,9 +462,9 @@ private[parquet] class ParquetFilters( canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { // Decimal type must make sure that filter value's scale matched the file. // If doesn't matched, which would cause data corruption. - def isDecimalMatched(value: Any, decimalMeta: DecimalMetadata): Boolean = value match { + def isDecimalMatched(value: Any, dec: DecimalLogicalTypeAnnotation): Boolean = value match { case decimal: JBigDecimal => - decimal.scale == decimalMeta.getScale + decimal.scale == dec.getScale case _ => false } @@ -430,14 +480,16 @@ private[parquet] class ParquetFilters( case ParquetStringType => value.isInstanceOf[String] case ParquetBinaryType => value.isInstanceOf[Array[Byte]] case ParquetDateType => value.isInstanceOf[Date] - case ParquetTimestampMicrosType | ParquetTimestampMillisType => - value.isInstanceOf[Timestamp] - case ParquetSchemaType(DECIMAL, INT32, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) - case ParquetSchemaType(DECIMAL, INT64, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) + case ParquetSchemaType(_, _class, INT64, _) + if _class == classOf[TimestampLogicalTypeAnnotation] => + value.isInstanceOf[Timestamp] + case ParquetSchemaType(decimal, _class, INT32, _) + if _class == classOf[DecimalLogicalTypeAnnotation] => + isDecimalMatched(value, decimal.asInstanceOf[DecimalLogicalTypeAnnotation]) + case ParquetSchemaType(decimal, _class, INT64, _) => + isDecimalMatched(value, decimal.asInstanceOf[DecimalLogicalTypeAnnotation]) + case ParquetSchemaType(decimal, _class, FIXED_LEN_BYTE_ARRAY, _) => + isDecimalMatched(value, decimal.asInstanceOf[DecimalLogicalTypeAnnotation]) case _ => false }) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 3319e73f2b31..a6e79337a27e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -26,6 +26,7 @@ import org.apache.parquet.hadoop.api.{InitContext, ReadSupport} import org.apache.parquet.hadoop.api.ReadSupport.ReadContext import org.apache.parquet.io.api.RecordMaterializer import org.apache.parquet.schema._ +import org.apache.parquet.schema.LogicalTypeAnnotation.{listType, ListLogicalTypeAnnotation} import org.apache.parquet.schema.Type.Repetition import org.apache.spark.internal.Logging @@ -49,7 +50,8 @@ import org.apache.spark.sql.types._ * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. */ -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) +private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], + val sessionLocalTz: TimeZone) extends ReadSupport[UnsafeRow] with Logging { private var catalystRequestedSchema: StructType = _ @@ -57,7 +59,7 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) // We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only // used in the vectorized reader, where we get the convertTz value directly, and the value here // is ignored. - this(None) + this(None, null) } /** @@ -107,7 +109,8 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) parquetRequestedSchema, ParquetReadSupport.expandUDT(catalystRequestedSchema), new ParquetToSparkSchemaConverter(conf), - convertTz) + convertTz, + sessionLocalTz) } } @@ -183,11 +186,12 @@ private[parquet] object ParquetReadSupport { // Unannotated repeated group should be interpreted as required list of required element, so // list element type is just the group itself. Clip it. - if (parquetList.getOriginalType == null && parquetList.isRepetition(Repetition.REPEATED)) { + if (parquetList.getLogicalTypeAnnotation == null + && parquetList.isRepetition(Repetition.REPEATED)) { clipParquetType(parquetList, elementType, caseSensitive) } else { assert( - parquetList.getOriginalType == OriginalType.LIST, + parquetList.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation], "Invalid Parquet schema. " + "Original type of annotated Parquet lists must be LIST: " + parquetList.toString) @@ -215,7 +219,7 @@ private[parquet] object ParquetReadSupport { ) { Types .buildGroup(parquetList.getRepetition) - .as(OriginalType.LIST) + .as(listType()) .addField(clipParquetType(repeatedGroup, elementType, caseSensitive)) .named(parquetList.getName) } else { @@ -223,7 +227,7 @@ private[parquet] object ParquetReadSupport { // repetition. Types .buildGroup(parquetList.getRepetition) - .as(OriginalType.LIST) + .as(listType()) .addField( Types .repeatedGroup() @@ -254,14 +258,14 @@ private[parquet] object ParquetReadSupport { val clippedRepeatedGroup = Types .repeatedGroup() - .as(repeatedGroup.getOriginalType) + .as(repeatedGroup.getLogicalTypeAnnotation) .addField(clipParquetType(parquetKeyType, keyType, caseSensitive)) .addField(clipParquetType(parquetValueType, valueType, caseSensitive)) .named(repeatedGroup.getName) Types .buildGroup(parquetMap.getRepetition) - .as(parquetMap.getOriginalType) + .as(parquetMap.getLogicalTypeAnnotation) .addField(clippedRepeatedGroup) .named(parquetMap.getName) } @@ -279,7 +283,7 @@ private[parquet] object ParquetReadSupport { val clippedParquetFields = clipParquetGroupFields(parquetRecord, structType, caseSensitive) Types .buildGroup(parquetRecord.getRepetition) - .as(parquetRecord.getOriginalType) + .as(parquetRecord.getLogicalTypeAnnotation) .addFields(clippedParquetFields: _*) .named(parquetRecord.getName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala index b2459dd0e8bb..4941de5b118b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRecordMaterializer.scala @@ -36,11 +36,13 @@ private[parquet] class ParquetRecordMaterializer( parquetSchema: MessageType, catalystSchema: StructType, schemaConverter: ParquetToSparkSchemaConverter, - convertTz: Option[TimeZone]) + convertTz: Option[TimeZone], + sessionLocalTz: TimeZone) extends RecordMaterializer[UnsafeRow] { private val rootConverter = - new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, convertTz, NoopUpdater) + new ParquetRowConverter(schemaConverter, parquetSchema, catalystSchema, convertTz, + sessionLocalTz, NoopUpdater) override def getCurrentRecord: UnsafeRow = rootConverter.currentRecord diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 004a96d13413..165d41fe9e00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -26,9 +26,9 @@ import scala.collection.mutable.ArrayBuffer import org.apache.parquet.column.Dictionary import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} -import org.apache.parquet.schema.{GroupType, MessageType, OriginalType, Type} -import org.apache.parquet.schema.OriginalType.{INT_32, LIST, UTF8} -import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.{BINARY, DOUBLE, FIXED_LEN_BYTE_ARRAY, INT32, INT64, INT96} +import org.apache.parquet.schema.{GroupType, MessageType, Type} +import org.apache.parquet.schema.LogicalTypeAnnotation._ +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow @@ -106,10 +106,10 @@ private[parquet] class ParquetPrimitiveConverter(val updater: ParentContainerUpd * 5 converters will be created: * * - a root [[ParquetRowConverter]] for [[MessageType]] `root`, which contains: - * - a [[ParquetPrimitiveConverter]] for required [[INT_32]] field `f1`, and + * - a [[ParquetPrimitiveConverter]] for required [[intType()]] field `f1`, and * - a nested [[ParquetRowConverter]] for optional [[GroupType]] `f2`, which contains: * - a [[ParquetPrimitiveConverter]] for required [[DOUBLE]] field `f21`, and - * - a [[ParquetStringConverter]] for optional [[UTF8]] string field `f22` + * - a [[ParquetStringConverter]] for optional [[stringType()]] string field `f22` * * When used as a root converter, [[NoopUpdater]] should be used since root converters don't have * any "parent" container. @@ -126,6 +126,7 @@ private[parquet] class ParquetRowConverter( parquetType: GroupType, catalystType: StructType, convertTz: Option[TimeZone], + sessionLocalTz: TimeZone, updater: ParentContainerUpdater) extends ParquetGroupConverter(updater) with Logging { @@ -257,17 +258,31 @@ private[parquet] class ParquetRowConverter( case StringType => new ParquetStringConverter(updater) - case TimestampType if parquetType.getOriginalType == OriginalType.TIMESTAMP_MICROS => + case TimestampType + if parquetType.getLogicalTypeAnnotation.isInstanceOf[TimestampLogicalTypeAnnotation] && + parquetType.getLogicalTypeAnnotation.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit + == TimeUnit.MICROS => new ParquetPrimitiveConverter(updater) { override def addLong(value: Long): Unit = { - updater.setLong(value) + val utc = parquetType.getLogicalTypeAnnotation + .asInstanceOf[TimestampLogicalTypeAnnotation].isAdjustedToUTC + val adjTime = if (utc) value else DateTimeUtils.convertTz(value, sessionLocalTz, UTC) + updater.setLong(adjTime.asInstanceOf[Long]) } } - case TimestampType if parquetType.getOriginalType == OriginalType.TIMESTAMP_MILLIS => + case TimestampType + if parquetType.getLogicalTypeAnnotation.isInstanceOf[TimestampLogicalTypeAnnotation] && + parquetType.getLogicalTypeAnnotation.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit + == TimeUnit.MILLIS => new ParquetPrimitiveConverter(updater) { override def addLong(value: Long): Unit = { - updater.setLong(DateTimeUtils.fromMillis(value)) + val utc = parquetType.getLogicalTypeAnnotation + .asInstanceOf[TimestampLogicalTypeAnnotation].isAdjustedToUTC + val rawTime = DateTimeUtils.fromMillis(value) + val adjTime = if (utc) rawTime else DateTimeUtils.convertTz(rawTime, + sessionLocalTz, UTC) + updater.setLong(adjTime.asInstanceOf[Long]) } } @@ -301,7 +316,8 @@ private[parquet] class ParquetRowConverter( // A repeated field that is neither contained by a `LIST`- or `MAP`-annotated group nor // annotated by `LIST` or `MAP` should be interpreted as a required list of required // elements where the element type is the type of the field. - case t: ArrayType if parquetType.getOriginalType != LIST => + case t: ArrayType + if !parquetType.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation] => if (parquetType.isPrimitive) { new RepeatedPrimitiveConverter(parquetType, t.elementType, updater) } else { @@ -316,7 +332,8 @@ private[parquet] class ParquetRowConverter( case t: StructType => new ParquetRowConverter( - schemaConverter, parquetType.asGroupType(), t, convertTz, new ParentContainerUpdater { + schemaConverter, parquetType.asGroupType(), t, convertTz, sessionLocalTz, + new ParentContainerUpdater { override def set(value: Any): Unit = updater.set(value.asInstanceOf[InternalRow].copy()) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 8ce8a86d2f02..e821ce399e56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -21,7 +21,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.parquet.schema._ -import org.apache.parquet.schema.OriginalType._ +import org.apache.parquet.schema.LogicalTypeAnnotation._ import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.parquet.schema.Type.Repetition._ @@ -92,10 +92,10 @@ class ParquetToSparkSchemaConverter( private def convertPrimitiveField(field: PrimitiveType): DataType = { val typeName = field.getPrimitiveTypeName - val originalType = field.getOriginalType + val logicalTypeAnnotation = field.getLogicalTypeAnnotation def typeString = - if (originalType == null) s"$typeName" else s"$typeName ($originalType)" + if (logicalTypeAnnotation == null) s"$typeName" else s"$typeName ($logicalTypeAnnotation)" def typeNotSupported() = throw new AnalysisException(s"Parquet type not supported: $typeString") @@ -110,8 +110,9 @@ class ParquetToSparkSchemaConverter( // specified in field.getDecimalMetadata. This is useful when interpreting decimal types stored // as binaries with variable lengths. def makeDecimalType(maxPrecision: Int = -1): DecimalType = { - val precision = field.getDecimalMetadata.getPrecision - val scale = field.getDecimalMetadata.getScale + val decimalType = field.getLogicalTypeAnnotation.asInstanceOf[DecimalLogicalTypeAnnotation] + val precision = decimalType.getPrecision + val scale = decimalType.getScale ParquetSchemaConverter.checkConversionRequirement( maxPrecision == -1 || 1 <= precision && precision <= maxPrecision, @@ -128,26 +129,27 @@ class ParquetToSparkSchemaConverter( case DOUBLE => DoubleType case INT32 => - originalType match { - case INT_8 => ByteType - case INT_16 => ShortType - case INT_32 | null => IntegerType - case DATE => DateType - case DECIMAL => makeDecimalType(Decimal.MAX_INT_DIGITS) - case UINT_8 => typeNotSupported() - case UINT_16 => typeNotSupported() - case UINT_32 => typeNotSupported() - case TIME_MILLIS => typeNotImplemented() + logicalTypeAnnotation match { + case i: IntLogicalTypeAnnotation => + if (!i.isSigned) typeNotSupported() else i.getBitWidth match { + case 8 => ByteType + case 16 => ShortType + case 32 => IntegerType + } + case null => IntegerType + case _: DateLogicalTypeAnnotation => DateType + case _: DecimalLogicalTypeAnnotation => makeDecimalType(Decimal.MAX_INT_DIGITS) + case _: TimeLogicalTypeAnnotation => typeNotImplemented() case _ => illegalType() } case INT64 => - originalType match { - case INT_64 | null => LongType - case DECIMAL => makeDecimalType(Decimal.MAX_LONG_DIGITS) - case UINT_64 => typeNotSupported() - case TIMESTAMP_MICROS => TimestampType - case TIMESTAMP_MILLIS => TimestampType + logicalTypeAnnotation match { + case i: IntLogicalTypeAnnotation => + if (!i.isSigned) typeNotSupported() else LongType + case null => LongType + case _: DecimalLogicalTypeAnnotation => makeDecimalType(Decimal.MAX_LONG_DIGITS) + case _: TimestampLogicalTypeAnnotation => TimestampType case _ => illegalType() } @@ -159,19 +161,21 @@ class ParquetToSparkSchemaConverter( TimestampType case BINARY => - originalType match { - case UTF8 | ENUM | JSON => StringType + logicalTypeAnnotation match { + case _: StringLogicalTypeAnnotation | + _: EnumLogicalTypeAnnotation | _: JsonLogicalTypeAnnotation => StringType case null if assumeBinaryIsString => StringType case null => BinaryType - case BSON => BinaryType - case DECIMAL => makeDecimalType() + case _: BsonLogicalTypeAnnotation => BinaryType + case _: DecimalLogicalTypeAnnotation => makeDecimalType() case _ => illegalType() } case FIXED_LEN_BYTE_ARRAY => - originalType match { - case DECIMAL => makeDecimalType(Decimal.maxPrecisionForBytes(field.getTypeLength)) - case INTERVAL => typeNotImplemented() + logicalTypeAnnotation match { + case _: DecimalLogicalTypeAnnotation => + makeDecimalType(Decimal.maxPrecisionForBytes(field.getTypeLength)) + case _: IntervalLogicalTypeAnnotation => typeNotImplemented() case _ => illegalType() } @@ -180,7 +184,7 @@ class ParquetToSparkSchemaConverter( } private def convertGroupField(field: GroupType): DataType = { - Option(field.getOriginalType).fold(convert(field): DataType) { + Option(field.getLogicalTypeAnnotation).fold(convert(field): DataType) { // A Parquet list is represented as a 3-level structure: // // group (LIST) { @@ -194,7 +198,7 @@ class ParquetToSparkSchemaConverter( // we need to check whether the 2nd level or the 3rd level refers to list element type. // // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists - case LIST => + case _: ListLogicalTypeAnnotation => ParquetSchemaConverter.checkConversionRequirement( field.getFieldCount == 1, s"Invalid list type $field") @@ -214,7 +218,7 @@ class ParquetToSparkSchemaConverter( // `MAP_KEY_VALUE` is for backwards-compatibility // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules-1 // scalastyle:on - case MAP | MAP_KEY_VALUE => + case _: MapLogicalTypeAnnotation | _: MapKeyValueTypeAnnotation => ParquetSchemaConverter.checkConversionRequirement( field.getFieldCount == 1 && !field.getType(0).isPrimitive, s"Invalid map type: $field") @@ -346,10 +350,10 @@ class SparkToParquetSchemaConverter( Types.primitive(BOOLEAN, repetition).named(field.name) case ByteType => - Types.primitive(INT32, repetition).as(INT_8).named(field.name) + Types.primitive(INT32, repetition).as(intType(8, true)).named(field.name) case ShortType => - Types.primitive(INT32, repetition).as(INT_16).named(field.name) + Types.primitive(INT32, repetition).as(intType(16, true)).named(field.name) case IntegerType => Types.primitive(INT32, repetition).named(field.name) @@ -364,10 +368,10 @@ class SparkToParquetSchemaConverter( Types.primitive(DOUBLE, repetition).named(field.name) case StringType => - Types.primitive(BINARY, repetition).as(UTF8).named(field.name) + Types.primitive(BINARY, repetition).as(stringType()).named(field.name) case DateType => - Types.primitive(INT32, repetition).as(DATE).named(field.name) + Types.primitive(INT32, repetition).as(dateType()).named(field.name) // NOTE: Spark SQL can write timestamp values to Parquet using INT96, TIMESTAMP_MICROS or // TIMESTAMP_MILLIS. TIMESTAMP_MICROS is recommended but INT96 is the default to keep the @@ -388,9 +392,11 @@ class SparkToParquetSchemaConverter( case SQLConf.ParquetOutputTimestampType.INT96 => Types.primitive(INT96, repetition).named(field.name) case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS => - Types.primitive(INT64, repetition).as(TIMESTAMP_MICROS).named(field.name) + Types.primitive(INT64, repetition).as(timestampType(true, TimeUnit.MICROS)) + .named(field.name) case SQLConf.ParquetOutputTimestampType.TIMESTAMP_MILLIS => - Types.primitive(INT64, repetition).as(TIMESTAMP_MILLIS).named(field.name) + Types.primitive(INT64, repetition).as(timestampType(true, TimeUnit.MILLIS)) + .named(field.name) } case BinaryType => @@ -407,9 +413,7 @@ class SparkToParquetSchemaConverter( case DecimalType.Fixed(precision, scale) if writeLegacyParquetFormat => Types .primitive(FIXED_LEN_BYTE_ARRAY, repetition) - .as(DECIMAL) - .precision(precision) - .scale(scale) + .as(decimalType(scale, precision)) .length(Decimal.minBytesForPrecision(precision)) .named(field.name) @@ -422,9 +426,7 @@ class SparkToParquetSchemaConverter( if precision <= Decimal.MAX_INT_DIGITS && !writeLegacyParquetFormat => Types .primitive(INT32, repetition) - .as(DECIMAL) - .precision(precision) - .scale(scale) + .as(decimalType(scale, precision)) .named(field.name) // Uses INT64 for 1 <= precision <= 18 @@ -432,18 +434,14 @@ class SparkToParquetSchemaConverter( if precision <= Decimal.MAX_LONG_DIGITS && !writeLegacyParquetFormat => Types .primitive(INT64, repetition) - .as(DECIMAL) - .precision(precision) - .scale(scale) + .as(decimalType(scale, precision)) .named(field.name) // Uses FIXED_LEN_BYTE_ARRAY for all other precisions case DecimalType.Fixed(precision, scale) if !writeLegacyParquetFormat => Types .primitive(FIXED_LEN_BYTE_ARRAY, repetition) - .as(DECIMAL) - .precision(precision) - .scale(scale) + .as(decimalType(scale, precision)) .length(Decimal.minBytesForPrecision(precision)) .named(field.name) @@ -468,7 +466,7 @@ class SparkToParquetSchemaConverter( // `array` as its element name as below. Therefore, we build manually // the correct group type here via the builder. (See SPARK-16777) Types - .buildGroup(repetition).as(LIST) + .buildGroup(repetition).as(listType()) .addField(Types .buildGroup(REPEATED) // "array" is the name chosen by parquet-hive (1.7.0 and prior version) @@ -486,7 +484,7 @@ class SparkToParquetSchemaConverter( // Here too, we should not use `listOfElements`. (See SPARK-16777) Types - .buildGroup(repetition).as(LIST) + .buildGroup(repetition).as(listType()) // "array" is the name chosen by parquet-avro (1.7.0 and prior version) .addField(convertField(StructField("array", elementType, nullable), REPEATED)) .named(field.name) @@ -517,7 +515,7 @@ class SparkToParquetSchemaConverter( // } // } Types - .buildGroup(repetition).as(LIST) + .buildGroup(repetition).as(listType()) .addField( Types.repeatedGroup() .addField(convertField(StructField("element", elementType, containsNull))) @@ -532,7 +530,7 @@ class SparkToParquetSchemaConverter( // } // } Types - .buildGroup(repetition).as(MAP) + .buildGroup(repetition).as(mapType()) .addField( Types .repeatedGroup() diff --git a/sql/core/src/test/resources/test-data/timestamp_dictionary.parq b/sql/core/src/test/resources/test-data/timestamp_dictionary.parq new file mode 100644 index 0000000000000000000000000000000000000000..dda0a3e3657006c730bfe8ae471969d9e47e7f7f GIT binary patch literal 1209 zcmc&!yH4Xk6dgMQJ3I=Ab}WzN!Uc&y3Q#yBz7lKaGX&D`VMkk4fP7)tb8Da&$ z98q$T_$>ne4qndA??-2l_+&FktYb1r+(wls@fjnavS54W)V~(TZTD1!4xL6+Eg)U> zIW!19rb0gm{PtnTE#!9|=M5w7LJZA*Lc6VI56!1hAi7a2CjOMfQidp@9ud%d57h`W z*in%fRAmK3m^=nmHKN^Vzlx+bV!)UI#ek!ONb$E~P|OM#_=D5OVg4xEe+BTGK4fgN zl}bhov67rnhz#N73k06jzCfl1DUB!fiy9BOuQ8}(O5g!!1je*`ND|KzeE~>f(2tbH z^TgLQw6{r(AZFPRbz{}#!tyHXn|9UnD%`d>_gtHE*V?qblj@3NSF7tw Q>Eh;KYGZY4Vvu`DsX&$1>fj?dc5?6` zd;xs`2V>&utJDt)EttlkgM^UFIo~<=+>b9b4o@|L$U4cD2!;hucv-|8LP*3JcnYYP zcmsf!12`yvYK3ejWITZ3CGz*>*T(l~Gy?ey6pGesQ7wu~|IzO4?5lJ2@H<_MrqadW zU+LnS^kHLkwjC!qN2_gnz%F`bo4E#AL4?aAk+Ufx7#7Q}?KpPtyl>{S$B!Gb9QG@u z*2Yx5>qTrf$%4@IyP+@_2umqpKt)1evk@y`NtRj#UM8HBN4yloAe?}Oj^Gve+8+ZU z$1q{218+X3lXTIuoDRWfMaGAKo?hW_a zVMuIV8~)8L>u val reader = new VectorizedParquetRecordReader( - null, enableOffHeapColumnVector, vectorizedReaderBatchSize) + null, TimeZone.getDefault, enableOffHeapColumnVector, vectorizedReaderBatchSize) try { reader.initialize(p, ("id" :: Nil).asJava) val batch = reader.resultBatch() @@ -199,7 +200,7 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { files.map(_.asInstanceOf[String]).foreach { p => val reader = new VectorizedParquetRecordReader( - null, enableOffHeapColumnVector, vectorizedReaderBatchSize) + null, TimeZone.getDefault, enableOffHeapColumnVector, vectorizedReaderBatchSize) try { reader.initialize(p, ("id" :: Nil).asJava) val batch = reader.resultBatch() @@ -454,7 +455,7 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { var sum = 0 files.map(_.asInstanceOf[String]).foreach { p => val reader = new VectorizedParquetRecordReader( - null, enableOffHeapColumnVector, vectorizedReaderBatchSize) + null, TimeZone.getDefault, enableOffHeapColumnVector, vectorizedReaderBatchSize) try { reader.initialize(p, ("c1" :: "c2" :: Nil).asJava) val batch = reader.resultBatch() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala index db73bfa149aa..edebac4b2cac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.sql.execution.datasources.parquet +import java.util.TimeZone + import scala.collection.JavaConverters._ import org.apache.parquet.hadoop.ParquetOutputFormat @@ -42,7 +44,8 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) reader.initialize(file.asInstanceOf[String], null) val batch = reader.resultBatch() assert(reader.nextBatch()) @@ -69,7 +72,8 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) reader.initialize(file.asInstanceOf[String], null) val batch = reader.resultBatch() assert(reader.nextBatch()) @@ -100,7 +104,8 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) reader.initialize(file, null /* set columns to null to project all columns */) val column = reader.resultBatch().column(0) assert(reader.nextBatch()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 6b05b9c0f720..6e3b8b76e784 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.util.Locale +import java.util.{Locale, TimeZone} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -658,7 +658,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { { val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) try { reader.initialize(file, null) val result = mutable.ArrayBuffer.empty[(Int, String)] @@ -677,7 +678,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { { val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) try { reader.initialize(file, ("_2" :: Nil).asJava) val result = mutable.ArrayBuffer.empty[(String)] @@ -695,7 +697,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { { val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) try { reader.initialize(file, ("_2" :: "_1" :: Nil).asJava) val result = mutable.ArrayBuffer.empty[(String, Int)] @@ -714,7 +717,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { { val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) try { reader.initialize(file, List[String]().asJava) var result = 0 @@ -755,7 +759,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { val schema = StructType(StructField("pcol", dt) :: Nil) val conf = sqlContext.conf val vectorizedReader = new VectorizedParquetRecordReader( - null, conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) + null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + conf.parquetVectorizedReaderBatchSize) val partitionValues = new GenericInternalRow(Array(v)) val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index f06e1867151e..f29dec80e458 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -98,7 +98,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } } - test("parquet timestamp conversion") { + test("parquet Impala timestamp conversion") { // Make a table with one parquet file written by impala, and one parquet file written by spark. // We should only adjust the timestamps in the impala file, and only if the conf is set val impalaFile = "test-data/impala_timestamp.parq" @@ -204,4 +204,50 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } } } + + test("parquet timestamp read path") { + Seq("timestamp_plain", "timestamp_dictionary").foreach({ file => + val timestampPath = Thread.currentThread() + .getContextClassLoader.getResource("test-data/" + file + ".parq").toURI.getPath + val expectedPath = Thread.currentThread() + .getContextClassLoader.getResource("test-data/" + file + ".txt").toURI.getPath + + withTempPath {tableDir => + val textValues = spark.read + .option("header", false).option("inferSchema", true) + .option("delimiter", ";").csv(expectedPath).collect + val timestamps = textValues.map( + row => (row.getInt(0), + row.getTimestamp(1), + row.getTimestamp(2), + row.getTimestamp(3), + row.getTimestamp(4) + ) + ) + FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) + + Seq(false, true).foreach { vectorized => + withSQLConf( + (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) + ) { + val readBack = spark.read.parquet(tableDir.getAbsolutePath).collect + + val expected = timestamps.map(_.toString).sorted + val actual = readBack.map( + row => (row.getInt(0), + row.getTimestamp(1), + row.getTimestamp(2), + row.getTimestamp(3), + row.getTimestamp(4) + ) + ).map(_.toString).sorted + assert(readBack.size === expected.size) + withClue(s"vectorized = $vectorized, file = $file") { + assert(actual === expected) + } + } + } + } + }) + } } From f90d63e62f9db61751e6872c24f337553ab0cc36 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Tue, 2 Oct 2018 18:07:09 +0200 Subject: [PATCH 02/26] Generate file with epoch (called rawValue) value as id --- .../test-data/timestamp_dictionary.parq | Bin 1209 -> 1280 bytes .../test-data/timestamp_dictionary.txt | 8 ++++---- .../resources/test-data/timestamp_plain.parq | Bin 1099 -> 1155 bytes .../resources/test-data/timestamp_plain.txt | 4 ++-- .../ParquetInteroperabilitySuite.scala | 16 +++++++++++++--- 5 files changed, 19 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/resources/test-data/timestamp_dictionary.parq b/sql/core/src/test/resources/test-data/timestamp_dictionary.parq index dda0a3e3657006c730bfe8ae471969d9e47e7f7f..d6efb1a013ccc0ce491425b5f770676ad6fdba63 100644 GIT binary patch literal 1280 zcmc(f%TB^T6o#j?ff0=gahf*iq6;CI0HMgGCT!fXFiJ#SnP_c?P;EiW%|su8h0ow) zxY4+D!OpF(;Zmo*3PV*yi#Z6555?yJONYv;^<)J8vhjAz&PK-fLoO}!l z@Q}Y__(vuVOTU;nG3kym3INhEcLN~)#Eg%h6z4hriSYsGZ28+sqhJc4&yE4U7I_$b z_K%DKH)c#4?EwrpAK-_Xm{P<|a10nrxC#D^!QX~>2t-MU^c=H|owVIFu$69T zHWotoQnSx3420^r=3I1eJ7qg^7dtJp*^rgAl1VG6>h@-(THe`{6DJ+hGURMQRcgAS t=CGR2t6FWts3{p-#7b5x7EX!&E(;cu2#yBz7lKaGX&D`VMkk4fP7)tb8Da&$ z98q$T_$>ne4qndA??-2l_+&FktYb1r+(wls@fjnavS54W)V~(TZTD1!4xL6+Eg)U> zIW!19rb0gm{PtnTE#!9|=M5w7LJZA*Lc6VI56!1hAi7a2CjOMfQidp@9ud%d57h`W z*in%fRAmK3m^=nmHKN^Vzlx+bV!)UI#ek!ONb$E~P|OM#_=D5OVg4xEe+BTGK4fgN zl}bhov67rnhz#N73k06jzCfl1DUB!fiy9BOuQ8}(O5g!!1je*`ND|KzeE~>f(2tbH z^TgLQw6{r(AZFPRbz{}#!tyHXn|9UnD%`d>_gtHE*V?qblj@3NSF7tw Q>EC<72O0c}=W z#yHuRNseV1qm9JGjlvV-#V5~T;$b{8c`1_;%Mm7IL8V8`GNMo?aOP$v7v&d+mLwy& z;SuxX24-_+wFcJ7vCN80K&pY+f~kpZ@-AjSXCpHU!?ffS3zJj}Q&Wq?G|QAULxWWF kR70agbF(CKGYb>5G~?tHV@=5z83qPq#P9~_P*8*c09O%xS^xk5 delta 396 zcmZqXJk4Pd;22~m${;ErDxxFG0%UNAa>z)qFflMNNU$&hNihZuBpy(JLk1+wG%>(Q z07!#Dg~9%Sd;kCcpZHN`V!kpKVSBjPfsm_N*)<@gm|~Or8AaKdR2d|gGE*jRWUP&0 zl2m2jkOXl>nZ#Jcvc&S#IBf7ZPErQw0R~Z!tG=mgV!<3Y?WYd(CWJ62Sv?ODrL{np9b4|$@83qPq M#Bc`aSWpxI0PK`cl>h($ diff --git a/sql/core/src/test/resources/test-data/timestamp_plain.txt b/sql/core/src/test/resources/test-data/timestamp_plain.txt index a39e27e1eff6..e439d6745565 100644 --- a/sql/core/src/test/resources/test-data/timestamp_plain.txt +++ b/sql/core/src/test/resources/test-data/timestamp_plain.txt @@ -1,2 +1,2 @@ -1;1969-12-31 16:00:00.000;1970-01-01 00:00:00.000;1969-12-31 16:00:00.000;1970-01-01 00:00:00.000 -2;1965-06-01 05:43:39.000;1965-06-01 12:43:39.000;1965-06-01 05:43:39.000;1965-06-01 12:43:39.000 +-144674181;1965-06-01 05:43:39.000;1965-06-01 12:43:39.000;1965-06-01 05:43:39.000;1965-06-01 12:43:39.000 +0;1969-12-31 16:00:00.000;1970-01-01 00:00:00.000;1969-12-31 16:00:00.000;1970-01-01 00:00:00.000 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index f29dec80e458..3a90e2621893 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{LongType, StructField, StructType, TimestampType} class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedSQLContext { test("parquet files with different physical schemas but share the same logical schema") { @@ -212,12 +213,21 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS val expectedPath = Thread.currentThread() .getContextClassLoader.getResource("test-data/" + file + ".txt").toURI.getPath + val schema = StructType(Array( + StructField("rawValue", LongType, false), + StructField("millisUtc", TimestampType, false), + StructField("millisNonUtc", TimestampType, false), + StructField("microsUtc", TimestampType, false), + StructField("microsNonUtc", TimestampType, false))) + withTempPath {tableDir => val textValues = spark.read - .option("header", false).option("inferSchema", true) + .schema(schema) + .option("inferSchema", false) + .option("header", false) .option("delimiter", ";").csv(expectedPath).collect val timestamps = textValues.map( - row => (row.getInt(0), + row => (row.getLong(0), row.getTimestamp(1), row.getTimestamp(2), row.getTimestamp(3), @@ -234,7 +244,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS val expected = timestamps.map(_.toString).sorted val actual = readBack.map( - row => (row.getInt(0), + row => (row.getLong(0), row.getTimestamp(1), row.getTimestamp(2), row.getTimestamp(3), From 82a06bfda200401414ecad15cd57a97576967dcb Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Tue, 9 Oct 2018 15:21:44 +0200 Subject: [PATCH 03/26] Fix predicate pushdown - equals --- .../parquet/ParquetFileFormat.scala | 6 ++-- .../datasources/parquet/ParquetFilters.scala | 34 +++++++++++++----- .../test-data/timestamp_pushdown.parq | Bin 0 -> 2982 bytes .../test-data/timestamp_pushdown.txt | 9 +++++ .../parquet/ParquetFilterSuite.scala | 6 ++-- .../ParquetInteroperabilitySuite.scala | 34 ++++++++++++++++++ 6 files changed, 76 insertions(+), 13 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/timestamp_pushdown.parq create mode 100644 sql/core/src/test/resources/test-data/timestamp_pushdown.txt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index c95ef0bcc18f..18f561d248f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -363,6 +363,8 @@ class ParquetFileFormat null) val sharedConf = broadcastedHadoopConf.value.value + val sessionLocalTz = DateTimeUtils.getTimeZone( + sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)) lazy val footerFileMetaData = ParquetFileReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData @@ -370,7 +372,7 @@ class ParquetFileFormat val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema val parquetFilters = new ParquetFilters(pushDownDate, pushDownTimestamp, pushDownDecimal, - pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive, sessionLocalTz) filters // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` @@ -395,8 +397,6 @@ class ParquetFileFormat } else { None } - val sessionLocalTz = DateTimeUtils.getTimeZone( - sharedConf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key)) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index e59d7e68d0ef..fd092c404ab9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.lang.{Boolean => JBoolean, Double => JDouble, Float => JFloat, Long => JLong} import java.math.{BigDecimal => JBigDecimal} import java.sql.{Date, Timestamp} -import java.util.Locale +import java.util.{Locale, TimeZone} import scala.collection.JavaConverters.asScalaBufferConverter import scala.language.existentials @@ -47,7 +47,8 @@ private[parquet] class ParquetFilters( pushDownDecimal: Boolean, pushDownStartWith: Boolean, pushDownInFilterThreshold: Int, - caseSensitive: Boolean) { + caseSensitive: Boolean, + sessionLocalTz : TimeZone) { /** * Holds a single field information stored in the underlying parquet file. @@ -103,6 +104,10 @@ private[parquet] class ParquetFilters( Binary.fromConstantByteArray(fixedLengthBytes, 0, numBytes) } + val reverseAdjustMillis = (x: Any) => DateTimeUtils.toMillis(DateTimeUtils.convertTz( + DateTimeUtils.fromMillis(x.asInstanceOf[Timestamp].getTime), + DateTimeUtils.TimeZoneUTC, sessionLocalTz)).asInstanceOf[JLong] + private val makeEq: PartialFunction[ParquetSchemaType, (String, Any) => FilterPredicate] = { case ParquetBooleanType => (n: String, v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[JBoolean]) @@ -137,15 +142,28 @@ private[parquet] class ParquetFilters( case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.eq( - longColumn(n), - Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong]).orNull) + val timestampType = logicalType.asInstanceOf[TimestampLogicalTypeAnnotation] + (n: String, v: Any) => FilterApi.eq( + longColumn(n), + Option(v).map( + if (timestampType.isAdjustedToUTC) { + t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) + .asInstanceOf[JLong] + } else { + t => DateTimeUtils.convertTz( + DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]), + DateTimeUtils.TimeZoneUTC, sessionLocalTz).asInstanceOf[JLong] + }).orNull) } else { + val timestampType = logicalType.asInstanceOf[TimestampLogicalTypeAnnotation] (n: String, v: Any) => FilterApi.eq( longColumn(n), - Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) + Option(v).map( + if (timestampType.isAdjustedToUTC) { + _.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong] + } else { + reverseAdjustMillis + }).orNull) } case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => diff --git a/sql/core/src/test/resources/test-data/timestamp_pushdown.parq b/sql/core/src/test/resources/test-data/timestamp_pushdown.parq new file mode 100644 index 0000000000000000000000000000000000000000..527c7649fdd9168fc335b055de5277e545f5ae92 GIT binary patch literal 2982 zcmchZO-vI(7=~w?wrndUZR2islOAdU6%cBdpHi`hl6c@?5CKgzP3qE4Lv>4W`O|}j z0~ZPLV2DSI2^vHEA)YuI&Kx-rV>pp`AgJdlIT5@5n_|x;NSZR>72tw-S z@4XE)L%WF7^P%>!TE)Su53xc;6t7qgV6FTrZpqS?wQE)@Z6jh&U^CwaJ020UGEJ|D z2N=t*5-e?r4t^)CwC%#M;7vO4K8FnhW4GR68;D`tDp5;YK%h5+U8cjlT3R_{>{KVc zT=ddlwfrhkOIw2N)(~yu)n-0^vF{hXHpDV+m0)Sh(#s9YGP~`q8n&x)SWT7}V zY8w#Msu;;7d*w|*{v^c2^nYZx8bRj=Bph(DEy3YdKNSUU3aF0AG=@)2aF)F9a>yqR zg@Nz88ig;r9eDLjDfzPyTXrKdjpS3~oRz)hamYw^)e;82PWxmd3g_N?r+W`|tUP6S_TZV)aX+IH>)oK0%kxf4Je(EL_Jn(TGLpH+X<;(%+~ zdDMbWoJ;yT`OlKqgATc#o$J9y;kzLRUenIq5I%7(^6PvNu(FTC4!NG4$KeLyww)0r zv}l&*pcyY$3ebp8>t(1k!Do7T#(+Q>pV7^y^Kh=aY(^KLIa{hsN422{-_?9(^D=Igm{CC5yfJLT`tfMc*Lo N4_NU6FpB;|{|kGuMrr^6 literal 0 HcmV?d00001 diff --git a/sql/core/src/test/resources/test-data/timestamp_pushdown.txt b/sql/core/src/test/resources/test-data/timestamp_pushdown.txt new file mode 100644 index 000000000000..dc28f2488bab --- /dev/null +++ b/sql/core/src/test/resources/test-data/timestamp_pushdown.txt @@ -0,0 +1,9 @@ +42;1969-12-31 16:00:42.0;1970-01-01 00:00:42.0;1969-12-31 16:00:42.0;1970-01-01 00:00:42.0 +1509237900;2017-10-28 17:45:00.0;2017-10-29 00:45:00.0;2017-10-28 17:45:00.0;2017-10-29 00:45:00.0 +1509240660;2017-10-28 18:31:00.0;2017-10-29 01:31:00.0;2017-10-28 18:31:00.0;2017-10-29 01:31:00.0 +1509237000;2017-10-28 17:30:00.0;2017-10-29 00:30:00.0;2017-10-28 17:30:00.0;2017-10-29 00:30:00.0 +1509239700;2017-10-28 18:15:00.0;2017-10-29 01:15:00.0;2017-10-28 18:15:00.0;2017-10-29 01:15:00.0 +1512520200;2017-12-05 16:30:00.0;2017-12-06 00:30:00.0;2017-12-05 16:30:00.0;2017-12-06 00:30:00.0 +1509237900;2017-10-28 17:45:00.0;2017-10-29 00:45:00.0;2017-10-28 17:45:00.0;2017-10-29 00:45:00.0 +1509238800;2017-10-28 18:00:00.0;2017-10-29 01:00:00.0;2017-10-28 18:00:00.0;2017-10-29 01:00:00.0 +1509239400;2017-10-28 18:10:00.0;2017-10-29 01:10:00.0;2017-10-28 18:10:00.0;2017-10-29 01:10:00.0 \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 255f7db8d135..cc5fdbaceb60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -62,7 +63,8 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex private lazy val parquetFilters = new ParquetFilters(conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, conf.parquetFilterPushDownDecimal, conf.parquetFilterPushDownStringStartWith, - conf.parquetFilterPushDownInFilterThreshold, conf.caseSensitiveAnalysis) + conf.parquetFilterPushDownInFilterThreshold, conf.caseSensitiveAnalysis, + DateTimeUtils.defaultTimeZone()) override def beforeEach(): Unit = { super.beforeEach() @@ -1177,7 +1179,7 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex def createParquetFilter(caseSensitive: Boolean): ParquetFilters = { new ParquetFilters(conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, conf.parquetFilterPushDownDecimal, conf.parquetFilterPushDownStringStartWith, - conf.parquetFilterPushDownInFilterThreshold, caseSensitive) + conf.parquetFilterPushDownInFilterThreshold, caseSensitive, DateTimeUtils.defaultTimeZone()) } val caseSensitiveParquetFilters = createParquetFilter(caseSensitive = true) val caseInsensitiveParquetFilters = createParquetFilter(caseSensitive = false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 3a90e2621893..db95c8a90c66 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -260,4 +260,38 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } }) } + + test("parquet timestamp predicate pushdown") { + Seq("timestamp_pushdown").foreach({ file => + val timestampPath = Thread.currentThread() + .getContextClassLoader.getResource("test-data/" + file + ".parq").toURI.getPath + val textFile = Thread.currentThread() + .getContextClassLoader.getResource("test-data/" + file + ".txt").toURI.getPath + + withTempPath { tableDir => + val textValues = spark.read + .option("inferSchema", false) + .option("header", false) + .option("delimiter", ";").csv(textFile).collect + FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) + + Seq(false, true).foreach { vectorized => + withSQLConf( + (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) + ) { + Seq((1, "millisUtc"), (2, "millisNonUtc"), (3, "microsUtc"), (4, "microsNonUtc")) + .foreach(column => textValues.map(row => row.getString(column._1)).foreach(item => { + val readBack = spark.read.parquet(tableDir.getAbsolutePath) + .select(column._2).distinct().where(s"${column._2} = '$item'").collect + withClue(s"vectorized = $vectorized, file = $file") { + assert(readBack.length === 1) + assert(readBack(0).getTimestamp(0).toString === item) + } + }) + ) + } + } + } + }) + } } From 1b67c870004a05a08fb8ec93dde8819ae88984d7 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Wed, 10 Oct 2018 14:04:26 +0200 Subject: [PATCH 04/26] refactor --- .../test-data/timestamp_pushdown.parq | Bin 2982 -> 3800 bytes .../test-data/timestamp_pushdown.txt | 1 + .../ParquetInteroperabilitySuite.scala | 51 +++++++++--------- 3 files changed, 25 insertions(+), 27 deletions(-) diff --git a/sql/core/src/test/resources/test-data/timestamp_pushdown.parq b/sql/core/src/test/resources/test-data/timestamp_pushdown.parq index 527c7649fdd9168fc335b055de5277e545f5ae92..3242703b91b17abe91bdd216bbb907130a610f42 100644 GIT binary patch literal 3800 zcmcha!A}!G6vk&usS64z&|S7k4>f@b2(<--iX0&EKnzh7G(kzQv`eVAfCWSk8jfg) z2V%I2rW!*~6At3B@!-LO@xZ}&Fdm4Be}Ei~Z)W$+?lMJ8BQ_)q@4flW*Z1x8m4V)i zQH3a8#n-I3U?D4VK$AD8342*9l{6HHLXjN6ViEKC>|2R)G#OVEgKI0wFc>~CO#bTj zriZD)6;*?4jjxy*42xrb(A-U=OzK{$ zGj?DNpdw1d&J2Fb>!Z+dP4?y(k}<_7Wzv#Tsr%cz6}AP74i4ICS4!^QxbtkJ6l5%X zU2svY^xcb5s?^I$Bq?_K$5syZU=@>kFw}OtR=Mlg6{=8^<)elJRI3=JB_mk!=(tge zWlXFC+w;1s@ftI$=g@uT0mX_@hQ*R#*9I%avj2u9%iR@EMyX-s;+exbC_j9)UXUMSKY&MW|>(gs{J=ump|Qicp`O6ADu}|=$W`uP84@N zJYL!O{N;FhPB;E6%cRoj)Xb&XM16JtQdJ-TVu_JDVx-PZO`6$idloh(vQuoMk8W&s zY@{i3BV?=1tm^ak`$s}@Kymx)W#YHQC@$5dK9JOhQb_K$hN88ioy9q5MTtKE{UZwf zfD5fioNK)+8+b(mG7dp&Mag2+LC(22+0hgmJm)GFzEQ&AR!?o@jS?XL^-Q!kB=6>A zkZoCFgXi5v!qxYss*8Y`A4~Zm{Lz8~JnEXHzK|dd2jD>6cNE*z?|3-aVJ9Q6NueDN zka4Q+JCN<tws}RcA?s4;Lm7~9!tOhr?Xq{uIeE98Og6xG z%8P`nTOQRz?RTfFpO`d2Lt!se*$=JAwSE0?0UZVfE*cZ72V#-hdXX3>o}0ty|m-PlxBlKh&n!0zEp8?K|2BdrK(7{`rWj1v4$T~$j-$Y zH~`17CgY%{>cyx-{Tm+#7qs(@k9J%b+>|Uv9pr6=llQW7TPYU4T+87F?Od*<9Tz4z zvr~vVWal+b-pkIsRwUfCGoV$?XH&C!Hk_GC>gn)gJfmxe^xN^wbXq6c;B-8Db55Ua z$z+4K^z2M(YBCrNML&1izxl}qCZ0(3e#}mm|yB<4vG8P{{l^l;o l^iDn68t?2F>+FcNcZ|0sl5LIt5%@x2{~^ONAp`I~?Qa_T-Pr&D delta 555 zcmca1yG)!Vz%j^hb2{TPR#hQM8_5_o4jEAvNmT|8NePak#PYDjoYGWLCNVa#Rg7XQ z7}Yo?GjdvRZeWBenk>jI!nTT0Z3CmtWCbQ|QArt51|Vc&U|>+Y#RO9{hf|9A786*7 z9)2fy;uk3+5xBW5t=fIBX`LXGV4D4o;Z)+1yghJ2=5AP+a - val timestampPath = Thread.currentThread() - .getContextClassLoader.getResource("test-data/" + file + ".parq").toURI.getPath - val textFile = Thread.currentThread() - .getContextClassLoader.getResource("test-data/" + file + ".txt").toURI.getPath + val timestampPath = Thread.currentThread() + .getContextClassLoader.getResource("test-data/timestamp_pushdown.parq").toURI.getPath + val textFile = Thread.currentThread() + .getContextClassLoader.getResource("test-data/timestamp_pushdown.txt").toURI.getPath - withTempPath { tableDir => - val textValues = spark.read - .option("inferSchema", false) - .option("header", false) - .option("delimiter", ";").csv(textFile).collect - FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) + withTempPath { tableDir => + val textValues = spark.read + .option("inferSchema", false) + .option("header", false) + .option("delimiter", ";").csv(textFile).collect + FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) - Seq(false, true).foreach { vectorized => - withSQLConf( - (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) - ) { - Seq((1, "millisUtc"), (2, "millisNonUtc"), (3, "microsUtc"), (4, "microsNonUtc")) - .foreach(column => textValues.map(row => row.getString(column._1)).foreach(item => { - val readBack = spark.read.parquet(tableDir.getAbsolutePath) - .select(column._2).distinct().where(s"${column._2} = '$item'").collect - withClue(s"vectorized = $vectorized, file = $file") { - assert(readBack.length === 1) - assert(readBack(0).getTimestamp(0).toString === item) - } - }) - ) - } + Seq(false, true).foreach { vectorized => + withSQLConf( + (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) + ) { + Seq((1, "millisUtc"), (2, "millisNonUtc"), (3, "microsUtc"), (4, "microsNonUtc")) + .foreach(column => textValues.map(row => row.getString(column._1)).foreach(item => { + val readBack = spark.read.parquet(tableDir.getAbsolutePath) + .select(column._2).distinct().where(s"${column._2} = '$item'").collect + withClue(s"vectorized = $vectorized, column = ${column._2}, item = $item") { + assert(readBack.length === 1) + assert(readBack(0).getTimestamp(0).toString === item) + } + })) } } - }) + } } } From dc8c3bbe9e18932327e68af278f34279f1efc89e Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Thu, 11 Oct 2018 18:08:41 +0200 Subject: [PATCH 05/26] Parquet pushdown - without lt and lteq predicates --- .../datasources/parquet/ParquetFilters.scala | 97 +++++++----------- .../test-data/timestamp_pushdown.parq | Bin 3800 -> 6683 bytes .../test-data/timestamp_pushdown.txt | 10 -- .../ParquetInteroperabilitySuite.scala | 54 +++++++--- 4 files changed, 74 insertions(+), 87 deletions(-) delete mode 100644 sql/core/src/test/resources/test-data/timestamp_pushdown.txt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index fd092c404ab9..73e0e5c85862 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -104,9 +104,27 @@ private[parquet] class ParquetFilters( Binary.fromConstantByteArray(fixedLengthBytes, 0, numBytes) } - val reverseAdjustMillis = (x: Any) => DateTimeUtils.toMillis(DateTimeUtils.convertTz( - DateTimeUtils.fromMillis(x.asInstanceOf[Timestamp].getTime), - DateTimeUtils.TimeZoneUTC, sessionLocalTz)).asInstanceOf[JLong] + private def timestampValue(timestampType: TimestampLogicalTypeAnnotation, v: Any): JLong = + if (timestampType.getUnit == TimeUnit.MICROS) { + Option(v).map( + if (timestampType.isAdjustedToUTC) { + t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) + .asInstanceOf[JLong] + } else { + t => DateTimeUtils.convertTz( + DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]), + DateTimeUtils.TimeZoneUTC, sessionLocalTz).asInstanceOf[JLong] + }).orNull + } else { + Option(v).map( + if (timestampType.isAdjustedToUTC) { + _.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong] + } else { + t => DateTimeUtils.toMillis(DateTimeUtils.convertTz( + DateTimeUtils.fromMillis(t.asInstanceOf[Timestamp].getTime), + DateTimeUtils.TimeZoneUTC, sessionLocalTz)).asInstanceOf[JLong] + }).orNull + } private val makeEq: PartialFunction[ParquetSchemaType, (String, Any) => FilterPredicate] = { case ParquetBooleanType => @@ -141,30 +159,9 @@ private[parquet] class ParquetFilters( Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - val timestampType = logicalType.asInstanceOf[TimestampLogicalTypeAnnotation] - (n: String, v: Any) => FilterApi.eq( - longColumn(n), - Option(v).map( - if (timestampType.isAdjustedToUTC) { - t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong] - } else { - t => DateTimeUtils.convertTz( - DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]), - DateTimeUtils.TimeZoneUTC, sessionLocalTz).asInstanceOf[JLong] - }).orNull) - } else { - val timestampType = logicalType.asInstanceOf[TimestampLogicalTypeAnnotation] - (n: String, v: Any) => FilterApi.eq( - longColumn(n), - Option(v).map( - if (timestampType.isAdjustedToUTC) { - _.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong] - } else { - reverseAdjustMillis - }).orNull) - } + (n: String, v: Any) => FilterApi.eq( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.eq( @@ -210,18 +207,10 @@ private[parquet] class ParquetFilters( Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.notEq( - longColumn(n), - Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong]).orNull) - } else { - (n: String, v: Any) => - FilterApi.notEq( - longColumn(n), - Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]).orNull) - } + (n: String, v: Any) => + FilterApi.notEq( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.notEq( @@ -356,17 +345,10 @@ private[parquet] class ParquetFilters( FilterApi.gt(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.gt( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - } else { - (n: String, v: Any) => - FilterApi.gt( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - } + (n: String, v: Any) => + FilterApi.gt( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => @@ -403,17 +385,10 @@ private[parquet] class ParquetFilters( FilterApi.gtEq(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.gtEq( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - } else { - (n: String, v: Any) => - FilterApi.gtEq( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - } + (n: String, v: Any) => + FilterApi.gtEq( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => diff --git a/sql/core/src/test/resources/test-data/timestamp_pushdown.parq b/sql/core/src/test/resources/test-data/timestamp_pushdown.parq index 3242703b91b17abe91bdd216bbb907130a610f42..3b4c859f1ff86ce3dc200c2d11cc9fa0abbdc4e4 100644 GIT binary patch literal 6683 zcmchcUuYCZ9LHzZ{JFFjNy6>AYfe4UY!l6$yO-SM?ixyjQpBbYH=$UMqYcGkt zL~To3f{4%(e2@@Ik^X@+l+ZsSf_)01k9qJ(#HS)6Vto)FTF66@{$_S(cJ^j3(W@Gc zIQjj4^O^7L{C+$4!-2gAEd?n}N~BYfVGvS6Q7v@!1R_7)NJZ@dRtO43X|Ra7x&Lg0VGJF6R)V4CXNFmR@6OdaqK1yJ8ame4h^V1vQ4RHY6CMwfRMOldR;+A~88pHwPgYQbh$AlOkM9(JG~Q{Z>tBCw@3v}A&{Fmt&U=J0D_=0GjXyQUT^9jL`K zmuc}FJ}sU(NQ>tMEvAy_RO(>hl)>RW|GW!xAb)q^f8YlnwP*#0R+tmC1EkkULI6rL%D> z8Sk=U&QW(K6DCN0-7Sy+ucYteVLiF-6u@57QH*h6U!dUA3JS6@)8ZABReG6%0I#IC zGX>e0q=u;gh6Mq$l5$q^DTr{BvcUpK<+%d(3-1W#mGqq+3;rNaAYn!jO2A`aY;wHa zo(3%x%Dr?1rwWX=JEee=MzrFPK^ii<+XzZQaOT^sHGvOv8xZ48=WkwrJnzbh0HB_XXe#TGH#}D zk#ynv2-glv0Vt^+Y6j6x?F4#X(NQ>)vr;)7Wy*N4p*2=c<7-e{8A^$n#g%z4UfJ?i z=6SI4HF)dL1p54oiZ`aBRbMt4s2wN z70Hz%u1E@?;)=Z2DUW~^M=IYFX2@HW><)~N0^jxuFHgpSHuLkI6E^KMmH+! z7x%q5S~z7Nog5mq$MQ-OQfk`ZZCT2nekWf%X*+*5j1`K-!sOdiLv1bnGtJQ`EQcId z*l``07I>!-XC&UxBGc~+!H zAt8{0zjqmP661Ogg#*t?K&JhZH^OAm>Lbs~jO=@+9DH7`7JhjHgL_B9kuPrm@?*DC z`~z98VPuf)ohS!iuBjG&tv0HJom2EB1cA@W>esSyt+v<7s$+z6uw@PIV!~a(&bSDQ$h=X)AFdnuf=Pvd~DBLrp0qqi08Ga=fkRu zpyuYnEEm`d1Vq)Ti<0`21a8O!+K#Q;$nlSTsgZ$wRY%B+lCjhXWZILh+s^Uidm9WHw<$i{<+pA8oo>$QB@=#)C?pm_E}m9e?Xs8cV>>WpkW zR5AUD+A|KW-VLjDIMx5v*Z~Ns{;LFTc=)u3TDJk?7BCh!Gq9NIKwjJoWZFi-Xzd=1 zC*N#k^HWvCH$!l>>JztY`#2_LR#^=)9)VBo-PUo~(-3QZFx zi&h_bp`DQ{W&1*Vweag53?7*6*E?`R3f&XQcEReCeP%Z!SIYJ?yQ_s?>SXB>nC*8Y z<5DM=GmqI`aT#OGCfE}GYzOvi!p|lz_H`O5p1rz7$eORD7%yJ7t$93lrlpD`A-l zD}Ly}-ho5=-aH&@KYp@M9F8T^8Ea%{IMZ!sQmIUSBs)A}nRbtDCG$P$<2~t2cY362 WXt=9GpMmBk@Poc@L}&p1Tj)RFtJ!svVwBV z-`H#>=L3Z@fkF#7CNuI$PM#ph!MuP2A$*E+@_rF%)-{Z3KvtlT1=BaK$$CPHASzJE zoLlS)n;09&o<%%Bdt{k_)Hz`b#tV~W1ZCLPF{)kQwVC{fO`A_rMw9^vnHU%t)c)~7 zZDIwf_{WE-VvPXMQaL6dbzj7S@z!KrA$eAy(p!Q+KG4o*P&+$>!FB?vJz^G2+eCnV zk_H*MO~hvM8BXoV^Tm0Xo`@n__CypRGf5m|3G*azo5>fsv?u=rDn1|q^qU;WhyxNZ dQEq0SeUlp{D to_timestamp('2017-10-29 00:45:00.0')" + // s"${column._2} >= to_timestamp('2017-10-29 00:45:00.0')" + // s"${column._2} != to_timestamp('1970-01-01 00:00:55.0')" test("parquet timestamp predicate pushdown") { val timestampPath = Thread.currentThread() .getContextClassLoader.getResource("test-data/timestamp_pushdown.parq").toURI.getPath - val textFile = Thread.currentThread() - .getContextClassLoader.getResource("test-data/timestamp_pushdown.txt").toURI.getPath + + def verifyPredicate(dataFrame: DataFrame, column: String, + item: String, predicate: String, vectorized: Boolean): Unit = { + val filter = s"$column $predicate to_timestamp('$item')" + withSQLConf( + (SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key, "true") + ) { + val withPushdown = dataFrame.where(filter).collect + withSQLConf( + (SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key, "false") + ) { + val withoutPushdown = dataFrame.where(filter).collect + withClue(s"vectorized = $vectorized, column = ${column}, item = $item") { + assert(withPushdown === withoutPushdown) + } + } + } + } withTempPath { tableDir => - val textValues = spark.read - .option("inferSchema", false) - .option("header", false) - .option("delimiter", ";").csv(textFile).collect FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) Seq(false, true).foreach { vectorized => withSQLConf( (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) ) { - Seq((1, "millisUtc"), (2, "millisNonUtc"), (3, "microsUtc"), (4, "microsNonUtc")) - .foreach(column => textValues.map(row => row.getString(column._1)).foreach(item => { - val readBack = spark.read.parquet(tableDir.getAbsolutePath) - .select(column._2).distinct().where(s"${column._2} = '$item'").collect - withClue(s"vectorized = $vectorized, column = ${column._2}, item = $item") { - assert(readBack.length === 1) - assert(readBack(0).getTimestamp(0).toString === item) - } - })) + val losAngeles = spark.read.parquet(tableDir.getAbsolutePath).select("inLosAngeles") + .collect.map(_.getString(0)) + val utc = spark.read.parquet(tableDir.getAbsolutePath).select("inUtc") + .collect.map(_.getString(0)) + val singapore = spark.read.parquet(tableDir.getAbsolutePath).select("inPerth") + .collect.map(_.getString(0)) + Seq(losAngeles, utc, singapore).foreach(values => values.foreach(item => + Seq("millisUtc", "millisNonUtc", "microsUtc", "microsNonUtc").foreach(column => { + val dataFrame = spark.read.parquet(tableDir.getAbsolutePath).select(column) + verifyPredicate(dataFrame, column, item, "=", vectorized) + verifyPredicate(dataFrame, column, item, "!=", vectorized) + verifyPredicate(dataFrame, column, item, ">", vectorized) + verifyPredicate(dataFrame, column, item, ">=", vectorized) + }) + )) } } } From 73e431b4786bbe80f4c2aa102294b0b59350e822 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 12 Oct 2018 13:59:33 +0200 Subject: [PATCH 06/26] Fix lt and lteq predicates with timestamps --- .../datasources/parquet/ParquetFilters.scala | 30 +++--------- .../ParquetInteroperabilitySuite.scala | 49 ++++++++++--------- 2 files changed, 34 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 73e0e5c85862..2279efebce90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -250,17 +250,10 @@ private[parquet] class ParquetFilters( FilterApi.lt(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.lt( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - } else { - (n: String, v: Any) => - FilterApi.lt( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - } + (n: String, v: Any) => + FilterApi.lt( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => @@ -298,17 +291,10 @@ private[parquet] class ParquetFilters( FilterApi.ltEq(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - if (logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == TimeUnit.MICROS) { - (n: String, v: Any) => - FilterApi.ltEq( - longColumn(n), - DateTimeUtils.fromJavaTimestamp(v.asInstanceOf[Timestamp]).asInstanceOf[JLong]) - } else { - (n: String, v: Any) => - FilterApi.ltEq( - longColumn(n), - v.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong]) - } + (n: String, v: Any) => + FilterApi.ltEq( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 8d220ba79264..b6c1d9f52e47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -18,15 +18,14 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File +import java.util.TimeZone import scala.language.existentials - import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName - import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf @@ -290,27 +289,31 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS withTempPath { tableDir => FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) - Seq(false, true).foreach { vectorized => - withSQLConf( - (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) - ) { - val losAngeles = spark.read.parquet(tableDir.getAbsolutePath).select("inLosAngeles") - .collect.map(_.getString(0)) - val utc = spark.read.parquet(tableDir.getAbsolutePath).select("inUtc") - .collect.map(_.getString(0)) - val singapore = spark.read.parquet(tableDir.getAbsolutePath).select("inPerth") - .collect.map(_.getString(0)) - Seq(losAngeles, utc, singapore).foreach(values => values.foreach(item => - Seq("millisUtc", "millisNonUtc", "microsUtc", "microsNonUtc").foreach(column => { - val dataFrame = spark.read.parquet(tableDir.getAbsolutePath).select(column) - verifyPredicate(dataFrame, column, item, "=", vectorized) - verifyPredicate(dataFrame, column, item, "!=", vectorized) - verifyPredicate(dataFrame, column, item, ">", vectorized) - verifyPredicate(dataFrame, column, item, ">=", vectorized) - }) - )) + Seq("America/Los_Angeles", "Australia/Perth").foreach({ timeZone => + TimeZone.setDefault(TimeZone.getTimeZone(timeZone)) + Seq(false, true).foreach { vectorized => + withSQLConf( + (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) + ) { + val losAngeles = spark.read.parquet(tableDir.getAbsolutePath).select("inLosAngeles") + .collect.map(_.getString(0)) + val utc = spark.read.parquet(tableDir.getAbsolutePath).select("inUtc") + .collect.map(_.getString(0)) + val singapore = spark.read.parquet(tableDir.getAbsolutePath).select("inPerth") + .collect.map(_.getString(0)) + Seq(losAngeles, utc, singapore).foreach(values => values.foreach(item => + Seq("millisUtc", "millisNonUtc", "microsUtc", "microsNonUtc").foreach(column => { + val dataFrame = spark.read.parquet(tableDir.getAbsolutePath).select(column) + verifyPredicate(dataFrame, column, item, "=", vectorized) + verifyPredicate(dataFrame, column, item, "!=", vectorized) + verifyPredicate(dataFrame, column, item, ">", vectorized) + verifyPredicate(dataFrame, column, item, ">=", vectorized) + verifyPredicate(dataFrame, column, item, "<", vectorized) + verifyPredicate(dataFrame, column, item, "<=", vectorized) + }) + )) + } } - } - } + })} } } From b6fe694f0df274a87324265f011a6f59a94c9fdc Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 12 Oct 2018 16:14:20 +0200 Subject: [PATCH 07/26] Fix style issues, remove whitespaces --- .../datasources/parquet/ParquetFilters.scala | 41 ++++++++----------- .../ParquetInteroperabilitySuite.scala | 2 + 2 files changed, 19 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 2279efebce90..86892dabdb84 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -130,10 +130,9 @@ private[parquet] class ParquetFilters( case ParquetBooleanType => (n: String, v: Any) => FilterApi.eq(booleanColumn(n), v.asInstanceOf[JBoolean]) case ParquetByteType | ParquetShortType | ParquetIntegerType => - (n: String, v: Any) => - FilterApi.eq( - intColumn(n), - Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) + (n: String, v: Any) => FilterApi.eq( + intColumn(n), + Option(v).map(_.asInstanceOf[Number].intValue.asInstanceOf[Integer]).orNull) case ParquetLongType => (n: String, v: Any) => FilterApi.eq(longColumn(n), v.asInstanceOf[JLong]) case ParquetFloatType => @@ -143,20 +142,17 @@ private[parquet] class ParquetFilters( // Binary.fromString and Binary.fromByteArray don't accept null values case ParquetStringType => - (n: String, v: Any) => - FilterApi.eq( - binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + (n: String, v: Any) => FilterApi.eq( + binaryColumn(n), + Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) case ParquetBinaryType => - (n: String, v: Any) => - FilterApi.eq( - binaryColumn(n), - Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + (n: String, v: Any) => FilterApi.eq( + binaryColumn(n), + Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) case ParquetDateType if pushDownDate => - (n: String, v: Any) => - FilterApi.eq( - intColumn(n), - Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) + (n: String, v: Any) => FilterApi.eq( + intColumn(n), + Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.eq( @@ -207,8 +203,7 @@ private[parquet] class ParquetFilters( Option(v).map(date => dateToDays(date.asInstanceOf[Date]).asInstanceOf[Integer]).orNull) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - (n: String, v: Any) => - FilterApi.notEq( + (n: String, v: Any) => FilterApi.notEq( longColumn(n), timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && @@ -250,8 +245,7 @@ private[parquet] class ParquetFilters( FilterApi.lt(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - (n: String, v: Any) => - FilterApi.lt( + (n: String, v: Any) => FilterApi.lt( longColumn(n), timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) @@ -291,10 +285,9 @@ private[parquet] class ParquetFilters( FilterApi.ltEq(intColumn(n), dateToDays(v.asInstanceOf[Date]).asInstanceOf[Integer]) case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => - (n: String, v: Any) => - FilterApi.ltEq( - longColumn(n), - timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) + (n: String, v: Any) => FilterApi.ltEq( + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => (n: String, v: Any) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index b6c1d9f52e47..a187090b54a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -21,11 +21,13 @@ import java.io.File import java.util.TimeZone import scala.language.existentials + import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER import org.apache.parquet.hadoop.ParquetFileReader import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName + import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf From 5fa5769d7c7db065893a1505128adf76f158ed82 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 1 Feb 2019 10:32:11 +0100 Subject: [PATCH 08/26] Fix failing tests --- .../sql/execution/datasources/parquet/ParquetFilters.scala | 2 +- .../datasources/parquet/ParquetInteroperabilitySuite.scala | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 86892dabdb84..604f29af942a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -594,7 +594,7 @@ private[parquet] class ParquetFilters( } override def keep(value: Binary): Boolean = { - UTF8String.fromBytes(value.getBytes).startsWith( + value != null && UTF8String.fromBytes(value.getBytes).startsWith( UTF8String.fromBytes(strToBinary.getBytes)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index a187090b54a8..3cda041734c4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -226,6 +226,7 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS .schema(schema) .option("inferSchema", false) .option("header", false) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss.SSS") .option("delimiter", ";").csv(expectedPath).collect val timestamps = textValues.map( row => (row.getLong(0), From cb6f06ca6e67ac2311234e5de2564c8b198404d3 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 1 Feb 2019 10:34:12 +0100 Subject: [PATCH 09/26] Upgrade to Parquet 1.11.0 release candidate --- dev/deps/spark-deps-hadoop-2.7 | 10 +++++----- dev/deps/spark-deps-hadoop-3.2 | 10 +++++----- pom.xml | 14 +++++++++++++- 3 files changed, 23 insertions(+), 11 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index cd4eed6e2937..c197920429f2 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -162,13 +162,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.10.1.jar -parquet-common-1.10.1.jar -parquet-encoding-1.10.1.jar +parquet-column-1.11.0.jar +parquet-common-1.11.0.jar +parquet-encoding-1.11.0.jar parquet-format-2.4.0.jar -parquet-hadoop-1.10.1.jar +parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.1.jar +parquet-jackson-1.11.0.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 6f3bbce3e746..27d0730e2e79 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -181,13 +181,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.10.1.jar -parquet-common-1.10.1.jar -parquet-encoding-1.10.1.jar +parquet-column-1.11.0.jar +parquet-common-1.11.0.jar +parquet-encoding-1.11.0.jar parquet-format-2.4.0.jar -parquet-hadoop-1.10.1.jar +parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.1.jar +parquet-jackson-1.11.0.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar diff --git a/pom.xml b/pom.xml index 19775050dee7..00cf808e16dc 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ 2.2.0 10.12.1.1 - 1.10.1 + 1.11.0 1.5.5 nohive com.twitter @@ -245,6 +245,18 @@ false + + parquet-1.11.0 + + Parquet 1.11.1 RC3 + https://repository.apache.org/content/repositories/orgapacheparquet-1020/ + + true + + + false + + From 45143868bdd344989f4e629d6c93ab95a300ff2a Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 1 Feb 2019 10:56:12 +0100 Subject: [PATCH 10/26] Fix name --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 00cf808e16dc..ee8f8062fa94 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,7 @@ parquet-1.11.0 - Parquet 1.11.1 RC3 + Parquet 1.11.0 RC3 https://repository.apache.org/content/repositories/orgapacheparquet-1020/ true From d93ecb8fddc9ef951ccada4289212c6b88a2f4a2 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 4 Feb 2019 18:00:21 +0100 Subject: [PATCH 11/26] Address code review comments --- .../parquet/VectorizedColumnReader.java | 8 ++-- .../parquet/ParquetFileFormat.scala | 6 ++- .../datasources/parquet/ParquetFilters.scala | 41 +++++++++---------- .../parquet/ParquetReadSupport.scala | 5 ++- .../parquet/ParquetRowConverter.scala | 30 ++++++++------ .../parquet/ParquetEncodingSuite.scala | 12 ++++-- 6 files changed, 56 insertions(+), 46 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java index 60da7cb4e2d9..69aaa6980e18 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java @@ -443,9 +443,7 @@ private void readLongBatch(int rowId, int num, WritableColumnVector column) thro for (int i = 0; i < num; i++) { if (defColumn.readInteger() == maxDefLevel) { long timestamp = dataColumn.readLong(); - if (needsTimezoneAdjustment()) { - timestamp = DateTimeUtils.convertTz(timestamp, sessionLocalTz, UTC); - } + timestamp = DateTimeUtils.convertTz(timestamp, sessionLocalTz, UTC); column.putLong(rowId + i, timestamp); } else { column.putNull(rowId + i); @@ -477,7 +475,9 @@ private boolean needsTimezoneAdjustment() { !((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).isAdjustedToUTC(); } - private boolean isTimestampWithUnit(LogicalTypeAnnotation logicalTypeAnnotation, LogicalTypeAnnotation.TimeUnit timeUnit) { + private boolean isTimestampWithUnit( + LogicalTypeAnnotation logicalTypeAnnotation, + LogicalTypeAnnotation.TimeUnit timeUnit) { return (logicalTypeAnnotation instanceof TimestampLogicalTypeAnnotation) && ((TimestampLogicalTypeAnnotation) logicalTypeAnnotation).getUnit() == timeUnit; } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 18f561d248f9..991dcef45b93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -410,8 +410,10 @@ class ParquetFileFormat val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { val vectorizedReader = new VectorizedParquetRecordReader( - convertTz.orNull, sessionLocalTz, - enableOffHeapColumnVector && taskContext.isDefined, capacity) + convertTz.orNull, + sessionLocalTz, + enableOffHeapColumnVector && taskContext.isDefined, + capacity) val iter = new RecordReaderIterator(vectorizedReader) // SPARK-23457 Register a task completion lister before `initialization`. taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index 604f29af942a..e66a4959c443 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -105,26 +105,23 @@ private[parquet] class ParquetFilters( } private def timestampValue(timestampType: TimestampLogicalTypeAnnotation, v: Any): JLong = - if (timestampType.getUnit == TimeUnit.MICROS) { - Option(v).map( - if (timestampType.isAdjustedToUTC) { - t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) - .asInstanceOf[JLong] - } else { - t => DateTimeUtils.convertTz( - DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]), - DateTimeUtils.TimeZoneUTC, sessionLocalTz).asInstanceOf[JLong] - }).orNull - } else { - Option(v).map( - if (timestampType.isAdjustedToUTC) { - _.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong] - } else { - t => DateTimeUtils.toMillis(DateTimeUtils.convertTz( - DateTimeUtils.fromMillis(t.asInstanceOf[Timestamp].getTime), - DateTimeUtils.TimeZoneUTC, sessionLocalTz)).asInstanceOf[JLong] - }).orNull - } + Option(v).map((timestampType.getUnit, timestampType.isAdjustedToUTC) match { + case (TimeUnit.MICROS, true) => + t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]) + .asInstanceOf[JLong] + case (TimeUnit.MICROS, false) => + t => DateTimeUtils.convertTz( + DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp]), + DateTimeUtils.TimeZoneUTC, sessionLocalTz).asInstanceOf[JLong] + case (TimeUnit.MILLIS, true) => + _.asInstanceOf[Timestamp].getTime.asInstanceOf[JLong] + case (TimeUnit.MILLIS, false) => + t => DateTimeUtils.toMillis(DateTimeUtils.convertTz( + DateTimeUtils.fromMillis(t.asInstanceOf[Timestamp].getTime), + DateTimeUtils.TimeZoneUTC, sessionLocalTz)).asInstanceOf[JLong] + case _ => throw new IllegalArgumentException(s"Unsupported timestamp type: " + + s"TIMESTAMP(${timestampType.getUnit}, ${timestampType.isAdjustedToUTC})") + }).orNull private val makeEq: PartialFunction[ParquetSchemaType, (String, Any) => FilterPredicate] = { case ParquetBooleanType => @@ -246,8 +243,8 @@ private[parquet] class ParquetFilters( case ParquetSchemaType(logicalType, _class, INT64, _) if pushDownTimestamp && _class == classOf[TimestampLogicalTypeAnnotation] => (n: String, v: Any) => FilterApi.lt( - longColumn(n), - timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) + longColumn(n), + timestampValue(logicalType.asInstanceOf[TimestampLogicalTypeAnnotation], v)) case ParquetSchemaType(_, _class, INT32, _) if pushDownDecimal && _class == classOf[DecimalLogicalTypeAnnotation] => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index a6e79337a27e..7873f533793a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -50,8 +50,9 @@ import org.apache.spark.sql.types._ * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. */ -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone], - val sessionLocalTz: TimeZone) +private[parquet] class ParquetReadSupport( + val convertTz: Option[TimeZone], + val sessionLocalTz: TimeZone) extends ReadSupport[UnsafeRow] with Logging { private var catalystRequestedSchema: StructType = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala index 165d41fe9e00..613d53c2d636 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala @@ -26,8 +26,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.parquet.column.Dictionary import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} -import org.apache.parquet.schema.{GroupType, MessageType, Type} -import org.apache.parquet.schema.LogicalTypeAnnotation._ +import org.apache.parquet.schema.{GroupType, LogicalTypeAnnotation, MessageType, Type} +import org.apache.parquet.schema.LogicalTypeAnnotation.{TimeUnit, _} import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName._ import org.apache.spark.internal.Logging @@ -210,6 +210,12 @@ private[parquet] class ParquetRowConverter( } } + def isTimestampWithUnit(parquetType: Type, timeUnit: LogicalTypeAnnotation.TimeUnit): Boolean = { + val logicalType = parquetType.getLogicalTypeAnnotation + logicalType.isInstanceOf[TimestampLogicalTypeAnnotation] && + logicalType.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit == timeUnit + } + /** * Creates a converter for the given Parquet type `parquetType` and Spark SQL data type * `catalystType`. Converted values are handled by `updater`. @@ -258,10 +264,7 @@ private[parquet] class ParquetRowConverter( case StringType => new ParquetStringConverter(updater) - case TimestampType - if parquetType.getLogicalTypeAnnotation.isInstanceOf[TimestampLogicalTypeAnnotation] && - parquetType.getLogicalTypeAnnotation.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit - == TimeUnit.MICROS => + case TimestampType if isTimestampWithUnit(parquetType, TimeUnit.MICROS) => new ParquetPrimitiveConverter(updater) { override def addLong(value: Long): Unit = { val utc = parquetType.getLogicalTypeAnnotation @@ -271,10 +274,7 @@ private[parquet] class ParquetRowConverter( } } - case TimestampType - if parquetType.getLogicalTypeAnnotation.isInstanceOf[TimestampLogicalTypeAnnotation] && - parquetType.getLogicalTypeAnnotation.asInstanceOf[TimestampLogicalTypeAnnotation].getUnit - == TimeUnit.MILLIS => + case TimestampType if isTimestampWithUnit(parquetType, TimeUnit.MILLIS) => new ParquetPrimitiveConverter(updater) { override def addLong(value: Long): Unit = { val utc = parquetType.getLogicalTypeAnnotation @@ -316,8 +316,8 @@ private[parquet] class ParquetRowConverter( // A repeated field that is neither contained by a `LIST`- or `MAP`-annotated group nor // annotated by `LIST` or `MAP` should be interpreted as a required list of required // elements where the element type is the type of the field. - case t: ArrayType - if !parquetType.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation] => + case t: ArrayType if + !parquetType.getLogicalTypeAnnotation.isInstanceOf[ListLogicalTypeAnnotation] => if (parquetType.isPrimitive) { new RepeatedPrimitiveConverter(parquetType, t.elementType, updater) } else { @@ -332,7 +332,11 @@ private[parquet] class ParquetRowConverter( case t: StructType => new ParquetRowConverter( - schemaConverter, parquetType.asGroupType(), t, convertTz, sessionLocalTz, + schemaConverter, + parquetType.asGroupType(), + t, + convertTz, + sessionLocalTz, new ParentContainerUpdater { override def set(value: Any): Unit = updater.set(value.asInstanceOf[InternalRow].copy()) }) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala index edebac4b2cac..d3aefed87c4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala @@ -44,7 +44,9 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + null, + TimeZone.getDefault, + conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) reader.initialize(file.asInstanceOf[String], null) val batch = reader.resultBatch() @@ -72,7 +74,9 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + null, + TimeZone.getDefault, + conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) reader.initialize(file.asInstanceOf[String], null) val batch = reader.resultBatch() @@ -104,7 +108,9 @@ class ParquetEncodingSuite extends ParquetCompatibilityTest with SharedSQLContex val conf = sqlContext.conf val reader = new VectorizedParquetRecordReader( - null, TimeZone.getDefault, conf.offHeapColumnVectorEnabled, + null, + TimeZone.getDefault, + conf.offHeapColumnVectorEnabled, conf.parquetVectorizedReaderBatchSize) reader.initialize(file, null /* set columns to null to project all columns */) val column = reader.resultBatch().column(0) From 21af765d30eb7cf8635370e35af0930639445bd7 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 8 Feb 2019 14:24:28 +0100 Subject: [PATCH 12/26] Add staging repository to use parquet-mr 1.11.0 RC3 --- pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/pom.xml b/pom.xml index ee8f8062fa94..2a06ffb443f2 100644 --- a/pom.xml +++ b/pom.xml @@ -257,6 +257,18 @@ false + + parquet-1.11.0 + + Parquet 1.11.0 RC3 + https://repository.apache.org/content/groups/staging/ + + true + + + false + + From 8d4b06c6d85a40644ff5b64de66df179ed45a304 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 8 Feb 2019 14:42:37 +0100 Subject: [PATCH 13/26] Update manifest --- dev/deps/spark-deps-hadoop-2.6 | 198 +++++++++++++++++++++++++++++++++ dev/deps/spark-deps-hadoop-2.7 | 3 +- dev/deps/spark-deps-hadoop-3.2 | 4 +- 3 files changed, 202 insertions(+), 3 deletions(-) create mode 100644 dev/deps/spark-deps-hadoop-2.6 diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 new file mode 100644 index 000000000000..8145d7833778 --- /dev/null +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -0,0 +1,198 @@ +JavaEWAH-0.3.2.jar +RoaringBitmap-0.5.11.jar +ST4-4.0.4.jar +activation-1.1.1.jar +aircompressor-0.10.jar +antlr-2.7.7.jar +antlr-runtime-3.4.jar +antlr4-runtime-4.7.jar +aopalliance-1.0.jar +aopalliance-repackaged-2.4.0-b34.jar +apache-log4j-extras-1.2.17.jar +apacheds-i18n-2.0.0-M15.jar +apacheds-kerberos-codec-2.0.0-M15.jar +api-asn1-api-1.0.0-M20.jar +api-util-1.0.0-M20.jar +arpack_combined_all-0.1.jar +arrow-format-0.10.0.jar +arrow-memory-0.10.0.jar +arrow-vector-0.10.0.jar +automaton-1.11-8.jar +avro-1.8.2.jar +avro-ipc-1.8.2.jar +avro-mapred-1.8.2-hadoop2.jar +bonecp-0.8.0.RELEASE.jar +breeze-macros_2.11-0.13.2.jar +breeze_2.11-0.13.2.jar +calcite-avatica-1.2.0-incubating.jar +calcite-core-1.2.0-incubating.jar +calcite-linq4j-1.2.0-incubating.jar +chill-java-0.9.3.jar +chill_2.11-0.9.3.jar +commons-beanutils-1.7.0.jar +commons-beanutils-core-1.8.0.jar +commons-cli-1.2.jar +commons-codec-1.10.jar +commons-collections-3.2.2.jar +commons-compiler-3.0.9.jar +commons-compress-1.8.1.jar +commons-configuration-1.6.jar +commons-crypto-1.0.0.jar +commons-dbcp-1.4.jar +commons-digester-1.8.jar +commons-httpclient-3.1.jar +commons-io-2.4.jar +commons-lang-2.6.jar +commons-lang3-3.5.jar +commons-logging-1.1.3.jar +commons-math3-3.4.1.jar +commons-net-3.1.jar +commons-pool-1.5.4.jar +compress-lzf-1.0.3.jar +core-1.1.2.jar +curator-client-2.6.0.jar +curator-framework-2.6.0.jar +curator-recipes-2.6.0.jar +datanucleus-api-jdo-3.2.6.jar +datanucleus-core-3.2.10.jar +datanucleus-rdbms-3.2.9.jar +derby-10.12.1.1.jar +eigenbase-properties-1.1.5.jar +flatbuffers-1.2.0-3f79e055.jar +generex-1.0.1.jar +gson-2.2.4.jar +guava-14.0.1.jar +guice-3.0.jar +guice-servlet-3.0.jar +hadoop-annotations-2.6.5.jar +hadoop-auth-2.6.5.jar +hadoop-client-2.6.5.jar +hadoop-common-2.6.5.jar +hadoop-hdfs-2.6.5.jar +hadoop-mapreduce-client-app-2.6.5.jar +hadoop-mapreduce-client-common-2.6.5.jar +hadoop-mapreduce-client-core-2.6.5.jar +hadoop-mapreduce-client-jobclient-2.6.5.jar +hadoop-mapreduce-client-shuffle-2.6.5.jar +hadoop-yarn-api-2.6.5.jar +hadoop-yarn-client-2.6.5.jar +hadoop-yarn-common-2.6.5.jar +hadoop-yarn-server-common-2.6.5.jar +hadoop-yarn-server-web-proxy-2.6.5.jar +hk2-api-2.4.0-b34.jar +hk2-locator-2.4.0-b34.jar +hk2-utils-2.4.0-b34.jar +hppc-0.7.2.jar +htrace-core-3.0.4.jar +httpclient-4.5.6.jar +httpcore-4.4.10.jar +ivy-2.4.0.jar +jackson-annotations-2.6.7.jar +jackson-core-2.6.7.jar +jackson-core-asl-1.9.13.jar +jackson-databind-2.6.7.1.jar +jackson-dataformat-yaml-2.6.7.jar +jackson-jaxrs-1.9.13.jar +jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.7.jar +jackson-module-paranamer-2.7.9.jar +jackson-module-scala_2.11-2.6.7.1.jar +jackson-xc-1.9.13.jar +janino-3.0.9.jar +javassist-3.18.1-GA.jar +javax.annotation-api-1.2.jar +javax.inject-1.jar +javax.inject-2.4.0-b34.jar +javax.servlet-api-3.1.0.jar +javax.ws.rs-api-2.0.1.jar +javolution-5.5.1.jar +jaxb-api-2.2.2.jar +jcl-over-slf4j-1.7.16.jar +jdo-api-3.0.1.jar +jersey-client-2.22.2.jar +jersey-common-2.22.2.jar +jersey-container-servlet-2.22.2.jar +jersey-container-servlet-core-2.22.2.jar +jersey-guava-2.22.2.jar +jersey-media-jaxb-2.22.2.jar +jersey-server-2.22.2.jar +jetty-6.1.26.jar +jetty-util-6.1.26.jar +jline-2.14.6.jar +joda-time-2.9.3.jar +jodd-core-3.5.2.jar +jpam-1.1.jar +json4s-ast_2.11-3.5.3.jar +json4s-core_2.11-3.5.3.jar +json4s-jackson_2.11-3.5.3.jar +json4s-scalap_2.11-3.5.3.jar +jsr305-1.3.9.jar +jta-1.1.jar +jtransforms-2.4.0.jar +jul-to-slf4j-1.7.16.jar +kryo-shaded-4.0.2.jar +kubernetes-client-3.0.0.jar +kubernetes-model-2.0.0.jar +leveldbjni-all-1.8.jar +libfb303-0.9.3.jar +libthrift-0.9.3.jar +log4j-1.2.17.jar +logging-interceptor-3.8.1.jar +lz4-java-1.4.0.jar +machinist_2.11-0.6.1.jar +macro-compat_2.11-1.1.1.jar +mesos-1.4.0-shaded-protobuf.jar +metrics-core-3.1.5.jar +metrics-graphite-3.1.5.jar +metrics-json-3.1.5.jar +metrics-jvm-3.1.5.jar +minlog-1.3.0.jar +netty-3.9.9.Final.jar +netty-all-4.1.17.Final.jar +objenesis-2.5.1.jar +okhttp-3.8.1.jar +okio-1.13.0.jar +opencsv-2.3.jar +orc-core-1.5.2-nohive.jar +orc-mapreduce-1.5.2-nohive.jar +orc-shims-1.5.2.jar +oro-2.0.8.jar +osgi-resource-locator-1.0.1.jar +paranamer-2.8.jar +parquet-column-1.10.1-SNAPSHOT.jar +parquet-common-1.10.1-SNAPSHOT.jar +parquet-encoding-1.10.1-SNAPSHOT.jar +parquet-format-2.5.1-SNAPSHOT.jar +parquet-hadoop-1.10.1-SNAPSHOT.jar +parquet-hadoop-bundle-1.6.0.jar +parquet-jackson-1.10.1-SNAPSHOT.jar +protobuf-java-2.5.0.jar +py4j-0.10.7.jar +pyrolite-4.13.jar +scala-compiler-2.11.12.jar +scala-library-2.11.12.jar +scala-parser-combinators_2.11-1.1.0.jar +scala-reflect-2.11.12.jar +scala-xml_2.11-1.0.5.jar +shapeless_2.11-2.3.2.jar +slf4j-api-1.7.16.jar +slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar +snappy-0.2.jar +snappy-java-1.1.7.1.jar +spire-macros_2.11-0.13.0.jar +spire_2.11-0.13.0.jar +stax-api-1.0-2.jar +stax-api-1.0.1.jar +stream-2.7.0.jar +stringtemplate-3.2.1.jar +super-csv-2.2.0.jar +univocity-parsers-2.7.3.jar +validation-api-1.1.0.Final.jar +xbean-asm6-shaded-4.8.jar +xercesImpl-2.9.1.jar +xmlenc-0.52.jar +xz-1.5.jar +zjsonpatch-0.3.0.jar +zookeeper-3.4.6.jar +zstd-jni-1.3.2-2.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index c197920429f2..8441bf87efe5 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -17,6 +17,7 @@ arpack_combined_all-0.1.jar arrow-format-0.12.0.jar arrow-memory-0.12.0.jar arrow-vector-0.12.0.jar +audience-annotations-0.7.0.jar automaton-1.11-8.jar avro-1.8.2.jar avro-ipc-1.8.2.jar @@ -165,7 +166,7 @@ paranamer-2.8.jar parquet-column-1.11.0.jar parquet-common-1.11.0.jar parquet-encoding-1.11.0.jar -parquet-format-2.4.0.jar +parquet-format-structures-1.11.0.jar parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.11.0.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 27d0730e2e79..30dfc0194a7b 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -15,7 +15,7 @@ arpack_combined_all-0.1.jar arrow-format-0.12.0.jar arrow-memory-0.12.0.jar arrow-vector-0.12.0.jar -audience-annotations-0.5.0.jar +audience-annotations-0.7.0.jar automaton-1.11-8.jar avro-1.8.2.jar avro-ipc-1.8.2.jar @@ -184,7 +184,7 @@ paranamer-2.8.jar parquet-column-1.11.0.jar parquet-common-1.11.0.jar parquet-encoding-1.11.0.jar -parquet-format-2.4.0.jar +parquet-format-structures-1.11.0.jar parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar parquet-jackson-1.11.0.jar From fbe3039c68a421222f37728e86f70980dfa9d70e Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Tue, 12 Feb 2019 15:15:07 +0100 Subject: [PATCH 14/26] Fix failing tests, address code review comments --- dev/deps/spark-deps-hadoop-2.6 | 198 ------------------ .../test-data/timestamp_dictionary.parq | Bin 1280 -> 1294 bytes .../resources/test-data/timestamp_plain.parq | Bin 1155 -> 1168 bytes .../test-data/timestamp_pushdown.parq | Bin 6683 -> 5001 bytes .../columnar/InMemoryColumnarQuerySuite.scala | 4 +- .../datasources/HadoopFsRelationSuite.scala | 2 +- 6 files changed, 3 insertions(+), 201 deletions(-) delete mode 100644 dev/deps/spark-deps-hadoop-2.6 diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 deleted file mode 100644 index 8145d7833778..000000000000 --- a/dev/deps/spark-deps-hadoop-2.6 +++ /dev/null @@ -1,198 +0,0 @@ -JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar -ST4-4.0.4.jar -activation-1.1.1.jar -aircompressor-0.10.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar -antlr4-runtime-4.7.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar -apache-log4j-extras-1.2.17.jar -apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api-1.0.0-M20.jar -api-util-1.0.0-M20.jar -arpack_combined_all-0.1.jar -arrow-format-0.10.0.jar -arrow-memory-0.10.0.jar -arrow-vector-0.10.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.11-0.13.2.jar -breeze_2.11-0.13.2.jar -calcite-avatica-1.2.0-incubating.jar -calcite-core-1.2.0-incubating.jar -calcite-linq4j-1.2.0-incubating.jar -chill-java-0.9.3.jar -chill_2.11-0.9.3.jar -commons-beanutils-1.7.0.jar -commons-beanutils-core-1.8.0.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.9.jar -commons-compress-1.8.1.jar -commons-configuration-1.6.jar -commons-crypto-1.0.0.jar -commons-dbcp-1.4.jar -commons-digester-1.8.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.5.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.6.0.jar -curator-framework-2.6.0.jar -curator-recipes-2.6.0.jar -datanucleus-api-jdo-3.2.6.jar -datanucleus-core-3.2.10.jar -datanucleus-rdbms-3.2.9.jar -derby-10.12.1.1.jar -eigenbase-properties-1.1.5.jar -flatbuffers-1.2.0-3f79e055.jar -generex-1.0.1.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-3.0.jar -guice-servlet-3.0.jar -hadoop-annotations-2.6.5.jar -hadoop-auth-2.6.5.jar -hadoop-client-2.6.5.jar -hadoop-common-2.6.5.jar -hadoop-hdfs-2.6.5.jar -hadoop-mapreduce-client-app-2.6.5.jar -hadoop-mapreduce-client-common-2.6.5.jar -hadoop-mapreduce-client-core-2.6.5.jar -hadoop-mapreduce-client-jobclient-2.6.5.jar -hadoop-mapreduce-client-shuffle-2.6.5.jar -hadoop-yarn-api-2.6.5.jar -hadoop-yarn-client-2.6.5.jar -hadoop-yarn-common-2.6.5.jar -hadoop-yarn-server-common-2.6.5.jar -hadoop-yarn-server-web-proxy-2.6.5.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar -hppc-0.7.2.jar -htrace-core-3.0.4.jar -httpclient-4.5.6.jar -httpcore-4.4.10.jar -ivy-2.4.0.jar -jackson-annotations-2.6.7.jar -jackson-core-2.6.7.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.6.7.1.jar -jackson-dataformat-yaml-2.6.7.jar -jackson-jaxrs-1.9.13.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.6.7.jar -jackson-module-paranamer-2.7.9.jar -jackson-module-scala_2.11-2.6.7.1.jar -jackson-xc-1.9.13.jar -janino-3.0.9.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar -javax.inject-1.jar -javax.inject-2.4.0-b34.jar -javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar -javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar -jetty-6.1.26.jar -jetty-util-6.1.26.jar -jline-2.14.6.jar -joda-time-2.9.3.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json4s-ast_2.11-3.5.3.jar -json4s-core_2.11-3.5.3.jar -json4s-jackson_2.11-3.5.3.jar -json4s-scalap_2.11-3.5.3.jar -jsr305-1.3.9.jar -jta-1.1.jar -jtransforms-2.4.0.jar -jul-to-slf4j-1.7.16.jar -kryo-shaded-4.0.2.jar -kubernetes-client-3.0.0.jar -kubernetes-model-2.0.0.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.9.3.jar -log4j-1.2.17.jar -logging-interceptor-3.8.1.jar -lz4-java-1.4.0.jar -machinist_2.11-0.6.1.jar -macro-compat_2.11-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-3.1.5.jar -metrics-graphite-3.1.5.jar -metrics-json-3.1.5.jar -metrics-jvm-3.1.5.jar -minlog-1.3.0.jar -netty-3.9.9.Final.jar -netty-all-4.1.17.Final.jar -objenesis-2.5.1.jar -okhttp-3.8.1.jar -okio-1.13.0.jar -opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar -parquet-column-1.10.1-SNAPSHOT.jar -parquet-common-1.10.1-SNAPSHOT.jar -parquet-encoding-1.10.1-SNAPSHOT.jar -parquet-format-2.5.1-SNAPSHOT.jar -parquet-hadoop-1.10.1-SNAPSHOT.jar -parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.1-SNAPSHOT.jar -protobuf-java-2.5.0.jar -py4j-0.10.7.jar -pyrolite-4.13.jar -scala-compiler-2.11.12.jar -scala-library-2.11.12.jar -scala-parser-combinators_2.11-1.1.0.jar -scala-reflect-2.11.12.jar -scala-xml_2.11-1.0.5.jar -shapeless_2.11-2.3.2.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.15.jar -snappy-0.2.jar -snappy-java-1.1.7.1.jar -spire-macros_2.11-0.13.0.jar -spire_2.11-0.13.0.jar -stax-api-1.0-2.jar -stax-api-1.0.1.jar -stream-2.7.0.jar -stringtemplate-3.2.1.jar -super-csv-2.2.0.jar -univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar -xbean-asm6-shaded-4.8.jar -xercesImpl-2.9.1.jar -xmlenc-0.52.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.6.jar -zstd-jni-1.3.2-2.jar diff --git a/sql/core/src/test/resources/test-data/timestamp_dictionary.parq b/sql/core/src/test/resources/test-data/timestamp_dictionary.parq index d6efb1a013ccc0ce491425b5f770676ad6fdba63..cef7a4f5d4c67a035f8137933c79c49f21c2fdfe 100644 GIT binary patch literal 1294 zcmWG=3^EjD5mgXX@c~jS3=9G`)kO*4{{sO76o@j2vWar(h;jfGaENj+Ffem4GV`Hn zsxa6eaE~sU4uo9I%B~^NdBkaw6l9W=;DGrb$b<(0lcX2}14J6;Q}U!?4#A-tx3r9y zo2U-Z5e$+tVjCFY%vnruW(zZ%`3}kyWs=PC;LgY_ON}p1&PdHo6y;(NWl?32;3!Hg z4@=A`O@(kdb2D>tGK)h?l2wI#dIV);z#a#yvuqb9#i3<$&6fi}JzRL_pef zP_%)h5ZWMeVjPk(k}+x=GNLS!svIo9P=Gl^lu3+3EJ-X&jl%{J?uf_`W6*%d211lU zQU{c3MVT0Yi4;UJh+SY46%m`oB5KE=Rs^>N;Xk-FMJU$5A{Esdm?#!&p0J6EiQNEO zbB7Ub4@zLct-8ZVnO(m?c1>didFdDv+%81m!!0_-gkljaF`;@2CW^&V6WD)FP zff;TQ&{K%0gxj@&nKHYUf$W;Z2K1EJD`qhcV6v7FEHBC|NiEXL%}+_q(aTHBO_g9z ztw_u*$Vp|8@G3|wDlAPc(akMVC`&CW&dkqKFw`?N)H6`fNGi?DNl~y!N;Nk#Pcbw~ yPEItkG)+#kv@o?uG&M3bH!@92HZd_sGDtQ{vozL}i~*)^WW=Ds%)k%;EC~V62?Z7a delta 322 zcmeC(q=xkG<;00!&0w{x>fq|KWk(qDuJ;nfLRUx0r z`b=ibsvj3|qwDhG?C z1W#^ePEKaAUw&R_NwO%D7>5`{Z4Wa@EzpE>ER#1f!%P5j&NG8VC$LSHVv&ZL3*>-B z#TKzoZevl-3ob}3DlAPc(akMVC`&CW&dkqKFw`?N&@dR{#J2 diff --git a/sql/core/src/test/resources/test-data/timestamp_plain.parq b/sql/core/src/test/resources/test-data/timestamp_plain.parq index 750d87be46009918771d57d749a4ab80f26371a8..e262a0cdbbdb92523be6ca57fa66e804917e1c04 100644 GIT binary patch literal 1168 zcmWG=3^EjD5LFOW(Gg_s*4i7{|5pFD8Qnm!eD>EJ&Kha2)UY-U4z#m zbS094Op+2DFc$)ua7QvpiZL)iq+zZjPa0+=4&At=WyCl{-GGi@kdzVggR`bE!kM=a z%mqvkQBfwz91re{%(B$@;^d6f+(c0>22mDO1__R$#PYDjoYGVXmoqmrCnvKwv?N(o z$frk8Mh5I}uu2|?lwW=xP&TJW2w4uOEx9NktW5-@O$S9AND841A}7WoDI*!9#vvoh zBB{#20Sp9~LqwUxSj3#f+|)R15W$X!3o!-_cw`_%fnfnmJ`AEv3=Cp(SVcv|o-v8q zF{t^$jYoJ4ZoVIi`LM`DH6JF5Z2kt2`Ae8V=1*aSTYwTga66_jl4;2iR#7prM_@~C zAuK_JGu(z-s5Zco4XQU_qR8I31+rlQ3&H;G%ZoBgQj7F*^HWlD^zssOQzh6_D-v@Ha#9&2yb2PF3QJQ#ER8iKV}OYl88NUjGcW`I%OU_C8vG0Z delta 371 zcmbQh*~}>t;22~m${?yBs-h#x0%UNAa>z`!WYXqjU=XmWE=u_Rf1XMISLwc4U@-iUK+D%x3BgY7MM#GkJ0|lZ*0;{qplrjcs5BsbyghYhs(s z!J?cOT##5)Seja*n_Hw%mReMtnV+X%sAp)PXQ&(O=NJ&|;UA))kyM(QlcHc`W?`6? zoMK^;YGG<>k(g$gl4fX-YMyFnlxS|2WNv0*VwPr{oMNmg86(5MfQ%U4FflNILK*=5 CU|g60 diff --git a/sql/core/src/test/resources/test-data/timestamp_pushdown.parq b/sql/core/src/test/resources/test-data/timestamp_pushdown.parq index 3b4c859f1ff86ce3dc200c2d11cc9fa0abbdc4e4..a7e72ad7663bc6ed64e5a4e7099f75946be7dcf3 100644 GIT binary patch delta 1222 zcmZ{kO=uHA6vuZa$!3#mKFnsOT@p<}TrG(;N!v7)CEAOKl@eKfE5TT`RzVQ4 zprEw}>w}^#R1r@F^-`)9(Vjer7QJ}zpb^Al^dwaZ`Zh_E`UU%z_x|s{Gw;U?JKb_+ zvy$rQjxvFHm@mvICK8h-cK9b=8INBOq`+d|r-z~-2!EKMkN71)P!BQe!8O9%Xo#_R zbrtBRVJKqkGHrn`rl36%W@32*EE{3Uyg`nzSWTQVhj78p@`4}D41AVmIBQA456gZy zD~8~8CC*zzaNZhpP7;={ecmkXhI<%|wobTh3qhB?94q!9?$c5;uGqtH(6JL~+j9_4PWi1Klib0{xbkVdtM+-NLXYK`k664i^{yhp=ypEp52i-_VcDVQ3{ zf)yG3(gNYPiH5~}Z*s#s>F@DHj+Pmsy_3MMB*`*t^DswAoF&?_O@|kj2;5X;`$a1k z4Td`og8{cn8|jjgpv#WR_-Sp~p~H$8a;X8lSOTMX6}2U~ijl_RZ^K<@1t+ax`oWi= zKb#euw1%mnymqbPTq7D5H(4@RaQ_DfEd{p@=cMRwgM9Vix%d$7Dd9JW_SK`qH>sJg znU#MQ%aE77RY**q@C1GDtwO%{>hKaT^sPFX&M9l)=9F4v%x6I1nWnWdpAK*E@_$Ev z!Do<_BnUJlOl|1cUpuETWMN9XGdHhbyd#;{cf9cq$3 z5@A%`IJDdT19jG2ishAkVJChEnAyCu;?gT|XhGYQcpL!(}BvEkm2!r^pVOnUJ=&-Z-q zd4D<0IkW!ThERL!PE8hMzZ|NQDGE^*V<0{x#)%C&ZfnKukqk9KbUduE552j zG3-(yPHJc@46W`}kabr;KS@aGI@!Loqxql~Z~W#~;XK)3s5m?!9&}BT27i)vRAZ>N zQLf%dY50nEp!&?C!V{{Znn2~Yl&eXv3Z08tNhRQW3!fSFX>ilhhHBWSowIWBz*i4# z){js%`&9rdAM^s%bhq)+Qj1;s79Xh>C;f_CIq-c5ivjzTk zgvC-5O;ptBZ7ZAztTfT`^sWsi#Hct**?FpN{kAT5oJt*xW7UiC+sTWX2_+}Ne^O$ zU6_p%Wzy5U{N}6ZGcTe=mrkB}jgxhi{+Mhn?dLq4?<({kE$wK>5$p$|ob$sMZk!xI z;3$6@W@vc!M-Xu%E<3wO*(s`^-700{b0Gu50A8$|q{>_fzLB!Ym&~Y`IPL5LWg}F1 z$>6oL%{41AJLmi>yv+F{;byl-r{k+RC*KA$G(TbXD9QUtvdO6`qf!QUg|ZV=nW}HfX8*B#Z+CKYM%Q|}y7h*fzGaKv*|W8~M@#1#bK2(4 g#?1c4jNXvxsqgBpuZ@i0hxC7c!gn4);9ux}09F2SN&o-= diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 0a1141c050ec..8f10b2cf105c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -506,7 +506,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { case plan: InMemoryRelation => plan }.head // InMemoryRelation's stats is file size before the underlying RDD is materialized - assert(inMemoryRelation.computeStats().sizeInBytes === 868) + assert(inMemoryRelation.computeStats().sizeInBytes === 894) // InMemoryRelation's stats is updated after materializing RDD dfFromFile.collect() @@ -519,7 +519,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { // Even CBO enabled, InMemoryRelation's stats keeps as the file size before table's stats // is calculated - assert(inMemoryRelation2.computeStats().sizeInBytes === 868) + assert(inMemoryRelation2.computeStats().sizeInBytes === 894) // InMemoryRelation's stats should be updated after calculating stats of the table // clear cache to simulate a fresh environment diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala index 6e08ee3c4ba3..da653dabf95c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala @@ -45,7 +45,7 @@ class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { import testImplicits._ Seq(1.0, 0.5).foreach { compressionFactor => withSQLConf("spark.sql.sources.fileCompressionFactor" -> compressionFactor.toString, - "spark.sql.autoBroadcastJoinThreshold" -> "434") { + "spark.sql.autoBroadcastJoinThreshold" -> "544") { withTempPath { workDir => // the file size is 740 bytes val workDirPath = workDir.getAbsolutePath From fb708c3c75492feaff8b50445c26643fa04c86ed Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 18 Feb 2019 14:43:09 +0100 Subject: [PATCH 15/26] Fix failing ParquetPartitionDiscoverySuite test ParquetPartitionDiscoverySuite failed when executed after ParquetInteroperabilitySuite using Maven The reasion for that is, that ParquetInteroperabilitySuite changes the timezone in one test case, but doesn't restore the original. This could be easily fixed by restoring the original timezone in a finally block. --- .../ParquetInteroperabilitySuite.scala | 54 +++++++++++-------- 1 file changed, 31 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index 3cda041734c4..33d6e343dc9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -289,34 +289,42 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS } } + def withTimeZone(timeZone: String)(f: => Unit): Unit = { + val tz = TimeZone.getDefault + TimeZone.setDefault(TimeZone.getTimeZone(timeZone)) + try f finally TimeZone.setDefault(tz) + } + withTempPath { tableDir => FileUtils.copyFile(new File(timestampPath), new File(tableDir, "part-00001.parq")) Seq("America/Los_Angeles", "Australia/Perth").foreach({ timeZone => - TimeZone.setDefault(TimeZone.getTimeZone(timeZone)) - Seq(false, true).foreach { vectorized => - withSQLConf( - (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) - ) { - val losAngeles = spark.read.parquet(tableDir.getAbsolutePath).select("inLosAngeles") - .collect.map(_.getString(0)) - val utc = spark.read.parquet(tableDir.getAbsolutePath).select("inUtc") - .collect.map(_.getString(0)) - val singapore = spark.read.parquet(tableDir.getAbsolutePath).select("inPerth") - .collect.map(_.getString(0)) - Seq(losAngeles, utc, singapore).foreach(values => values.foreach(item => - Seq("millisUtc", "millisNonUtc", "microsUtc", "microsNonUtc").foreach(column => { - val dataFrame = spark.read.parquet(tableDir.getAbsolutePath).select(column) - verifyPredicate(dataFrame, column, item, "=", vectorized) - verifyPredicate(dataFrame, column, item, "!=", vectorized) - verifyPredicate(dataFrame, column, item, ">", vectorized) - verifyPredicate(dataFrame, column, item, ">=", vectorized) - verifyPredicate(dataFrame, column, item, "<", vectorized) - verifyPredicate(dataFrame, column, item, "<=", vectorized) - }) - )) + withTimeZone(timeZone) { + Seq(false, true).foreach { vectorized => + withSQLConf( + (SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, vectorized.toString()) + ) { + val losAngeles = spark.read.parquet(tableDir.getAbsolutePath).select("inLosAngeles") + .collect.map(_.getString(0)) + val utc = spark.read.parquet(tableDir.getAbsolutePath).select("inUtc") + .collect.map(_.getString(0)) + val singapore = spark.read.parquet(tableDir.getAbsolutePath).select("inPerth") + .collect.map(_.getString(0)) + Seq(losAngeles, utc, singapore).foreach(values => values.foreach(item => + Seq("millisUtc", "millisNonUtc", "microsUtc", "microsNonUtc").foreach(column => { + val dataFrame = spark.read.parquet(tableDir.getAbsolutePath).select(column) + verifyPredicate(dataFrame, column, item, "=", vectorized) + verifyPredicate(dataFrame, column, item, "!=", vectorized) + verifyPredicate(dataFrame, column, item, ">", vectorized) + verifyPredicate(dataFrame, column, item, ">=", vectorized) + verifyPredicate(dataFrame, column, item, "<", vectorized) + verifyPredicate(dataFrame, column, item, "<=", vectorized) + }) + )) + } } } - })} + }) + } } } From 4deff14fbebe4536ea62857c86db51bd803cf1f0 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Wed, 20 Feb 2019 14:27:48 +0100 Subject: [PATCH 16/26] Use direct snapshot url --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2a06ffb443f2..55343b582bf0 100644 --- a/pom.xml +++ b/pom.xml @@ -261,7 +261,7 @@ parquet-1.11.0 Parquet 1.11.0 RC3 - https://repository.apache.org/content/groups/staging/ + https://repository.apache.org/content/repositories/orgapacheparquet-1023/ true From afc7564965a24fa6e1e2d9a8c97f6255faa231a2 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Thu, 28 Feb 2019 10:45:38 +0100 Subject: [PATCH 17/26] Change updatePolicy to always --- pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pom.xml b/pom.xml index 55343b582bf0..d47e69bc7fd6 100644 --- a/pom.xml +++ b/pom.xml @@ -264,9 +264,11 @@ https://repository.apache.org/content/repositories/orgapacheparquet-1023/ true + always false + always From 2998df2b813f94093dd3a2a1a35a0c2830b70967 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 4 Mar 2019 11:35:46 +0100 Subject: [PATCH 18/26] remove duplicated repository --- pom.xml | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/pom.xml b/pom.xml index d47e69bc7fd6..3363a68695b2 100644 --- a/pom.xml +++ b/pom.xml @@ -245,22 +245,10 @@ false - - parquet-1.11.0 - - Parquet 1.11.0 RC3 - https://repository.apache.org/content/repositories/orgapacheparquet-1020/ - - true - - - false - - parquet-1.11.0 - Parquet 1.11.0 RC3 + Parquet 1.11.0 RC4 https://repository.apache.org/content/repositories/orgapacheparquet-1023/ true From 440a9b3e3a0f9826fa5e27598e568cfb303ef644 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Tue, 5 Mar 2019 17:14:05 +0100 Subject: [PATCH 19/26] Use 1.12.0-SNAPSHOT --- pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 3363a68695b2..70151437817e 100644 --- a/pom.xml +++ b/pom.xml @@ -249,13 +249,13 @@ parquet-1.11.0 Parquet 1.11.0 RC4 - https://repository.apache.org/content/repositories/orgapacheparquet-1023/ + https://repository.apache.org/content/repositories/snapshots/ true always - false + true always From 6e95803ae9cc18c2b34bb7992357dca35755206e Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Thu, 7 Mar 2019 13:28:05 +0100 Subject: [PATCH 20/26] update deps file --- dev/deps/spark-deps-hadoop-2.7 | 12 ++++++------ dev/deps/spark-deps-hadoop-3.2 | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8441bf87efe5..62368f797627 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -163,13 +163,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.11.0.jar -parquet-common-1.11.0.jar -parquet-encoding-1.11.0.jar -parquet-format-structures-1.11.0.jar -parquet-hadoop-1.11.0.jar +parquet-column-1.12.0-SNAPSHOT.jar +parquet-common-1.12.0-SNAPSHOT.jar +parquet-encoding-1.12.0-SNAPSHOT.jar +parquet-format-structures-1.12.0-SNAPSHOT.jar +parquet-hadoop-1.12.0-SNAPSHOT.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.11.0.jar +parquet-jackson-1.12.0-SNAPSHOT.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 30dfc0194a7b..273db3b745ff 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -181,13 +181,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.11.0.jar -parquet-common-1.11.0.jar -parquet-encoding-1.11.0.jar -parquet-format-structures-1.11.0.jar -parquet-hadoop-1.11.0.jar +parquet-column-1.12.0-SNAPSHOT.jar +parquet-common-1.12.0-SNAPSHOT.jar +parquet-encoding-1.12.0-SNAPSHOT.jar +parquet-format-structures-1.12.0-SNAPSHOT.jar +parquet-hadoop-1.12.0-SNAPSHOT.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.11.0.jar +parquet-jackson-1.12.0-SNAPSHOT.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar From 9a39876fa88c6311ee52f8453c4ed0bbc18d553a Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 25 Mar 2019 10:53:02 +0100 Subject: [PATCH 21/26] fix manifest file --- dev/deps/spark-deps-hadoop-2.7 | 12 ++++++------ dev/deps/spark-deps-hadoop-3.2 | 14 +++++++------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 62368f797627..8441bf87efe5 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -163,13 +163,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.12.0-SNAPSHOT.jar -parquet-common-1.12.0-SNAPSHOT.jar -parquet-encoding-1.12.0-SNAPSHOT.jar -parquet-format-structures-1.12.0-SNAPSHOT.jar -parquet-hadoop-1.12.0-SNAPSHOT.jar +parquet-column-1.11.0.jar +parquet-common-1.11.0.jar +parquet-encoding-1.11.0.jar +parquet-format-structures-1.11.0.jar +parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.12.0-SNAPSHOT.jar +parquet-jackson-1.11.0.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 index 273db3b745ff..3f0a665c54ef 100644 --- a/dev/deps/spark-deps-hadoop-3.2 +++ b/dev/deps/spark-deps-hadoop-3.2 @@ -15,7 +15,7 @@ arpack_combined_all-0.1.jar arrow-format-0.12.0.jar arrow-memory-0.12.0.jar arrow-vector-0.12.0.jar -audience-annotations-0.7.0.jar +audience-annotations-0.5.0.jar automaton-1.11-8.jar avro-1.8.2.jar avro-ipc-1.8.2.jar @@ -181,13 +181,13 @@ orc-shims-1.5.5.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.12.0-SNAPSHOT.jar -parquet-common-1.12.0-SNAPSHOT.jar -parquet-encoding-1.12.0-SNAPSHOT.jar -parquet-format-structures-1.12.0-SNAPSHOT.jar -parquet-hadoop-1.12.0-SNAPSHOT.jar +parquet-column-1.11.0.jar +parquet-common-1.11.0.jar +parquet-encoding-1.11.0.jar +parquet-format-structures-1.11.0.jar +parquet-hadoop-1.11.0.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.12.0-SNAPSHOT.jar +parquet-jackson-1.11.0.jar protobuf-java-2.5.0.jar py4j-0.10.8.1.jar pyrolite-4.13.jar From ca9bdc31c9fcbb35b7c619f176916096f1c822bc Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 25 Mar 2019 16:32:47 +0100 Subject: [PATCH 22/26] purge Parquet artifacts from local repository --- dev/run-tests.py | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index dfad2991077b..9140e27ce306 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -294,7 +294,7 @@ def get_hadoop_profiles(hadoop_version): def build_spark_maven(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags - mvn_goals = ["clean", "package", "-DskipTests"] + mvn_goals = ["dependency:purge-local-repository", "-Dinclude=org.apache.parquet", "clean", "package", "-DskipTests"] profiles_and_goals = build_profiles + mvn_goals print("[info] Building Spark (w/Hive 1.2.1) using Maven with these arguments: ", diff --git a/pom.xml b/pom.xml index 70151437817e..5f6cd2f80f9d 100644 --- a/pom.xml +++ b/pom.xml @@ -249,7 +249,7 @@ parquet-1.11.0 Parquet 1.11.0 RC4 - https://repository.apache.org/content/repositories/snapshots/ + https://repository.apache.org/content/groups/staging/ true always From f8ecac15542d15805ea733781b501b4e6645e892 Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 25 Mar 2019 16:41:23 +0100 Subject: [PATCH 23/26] line too long --- dev/run-tests.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 9140e27ce306..2f3d30f1f09d 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -294,7 +294,8 @@ def get_hadoop_profiles(hadoop_version): def build_spark_maven(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags - mvn_goals = ["dependency:purge-local-repository", "-Dinclude=org.apache.parquet", "clean", "package", "-DskipTests"] + mvn_goals = ["dependency:purge-local-repository", "-Dinclude=org.apache.parquet", + "clean", "package", "-DskipTests"] profiles_and_goals = build_profiles + mvn_goals print("[info] Building Spark (w/Hive 1.2.1) using Maven with these arguments: ", From c9a50e4b21cb5e30fcfe3d9415625735f648f20b Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Mon, 25 Mar 2019 17:27:58 +0100 Subject: [PATCH 24/26] fix python style error --- dev/run-tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests.py b/dev/run-tests.py index 2f3d30f1f09d..c159f4d11ea2 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -295,7 +295,7 @@ def build_spark_maven(hadoop_version): # Enable all of the profiles for the build: build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags mvn_goals = ["dependency:purge-local-repository", "-Dinclude=org.apache.parquet", - "clean", "package", "-DskipTests"] + "clean", "package", "-DskipTests"] profiles_and_goals = build_profiles + mvn_goals print("[info] Building Spark (w/Hive 1.2.1) using Maven with these arguments: ", From 9dbf1523ed247dec11d21217241bfb6014dc39ee Mon Sep 17 00:00:00 2001 From: Nandor Kollar Date: Fri, 3 Jan 2020 16:19:04 +0100 Subject: [PATCH 25/26] Remove snapshot repo Parquet 1.11.0 is officially released, no need to use snapshot. --- pom.xml | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/pom.xml b/pom.xml index 5f6cd2f80f9d..8d2864b5b474 100644 --- a/pom.xml +++ b/pom.xml @@ -245,20 +245,6 @@ false - - parquet-1.11.0 - - Parquet 1.11.0 RC4 - https://repository.apache.org/content/groups/staging/ - - true - always - - - true - always - - From 06aaa647bafd6974be6186dc4cf98e807916e460 Mon Sep 17 00:00:00 2001 From: "H. Vetinari" Date: Sun, 28 Feb 2021 18:04:31 +0100 Subject: [PATCH 26/26] resolve merge conflicts, enough to compile at least sessionLocalTz and convertTz were doing the same thing; keep the version from master. --- dev/deps/spark-deps-hadoop-2.7 | 202 ----------- dev/deps/spark-deps-hadoop-3.2 | 221 ------------- dev/run-tests.py | 7 +- pom.xml | 13 - .../sql/catalyst/util/DateTimeUtils.scala | 2 + .../parquet/VectorizedColumnReader.java | 115 ++----- .../VectorizedParquetRecordReader.java | 15 +- .../parquet/ParquetFileFormat.scala | 48 +-- .../datasources/parquet/ParquetFilters.scala | 313 +++--------------- .../parquet/ParquetReadSupport.scala | 17 - .../parquet/ParquetRecordMaterializer.scala | 12 - .../parquet/ParquetRowConverter.scala | 51 +-- .../ParquetPartitionReaderFactory.scala | 30 +- .../v2/parquet/ParquetScanBuilder.scala | 5 +- .../benchmark/DataSourceReadBenchmark.scala | 13 - .../columnar/InMemoryColumnarQuerySuite.scala | 36 -- .../datasources/HadoopFsRelationSuite.scala | 83 ----- .../parquet/ParquetEncodingSuite.scala | 23 -- .../parquet/ParquetFilterSuite.scala | 24 +- .../datasources/parquet/ParquetIOSuite.scala | 27 +- .../ParquetInteroperabilitySuite.scala | 14 +- 21 files changed, 126 insertions(+), 1145 deletions(-) delete mode 100644 dev/deps/spark-deps-hadoop-2.7 delete mode 100644 dev/deps/spark-deps-hadoop-3.2 delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 deleted file mode 100644 index 8441bf87efe5..000000000000 --- a/dev/deps/spark-deps-hadoop-2.7 +++ /dev/null @@ -1,202 +0,0 @@ -JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar -ST4-4.0.4.jar -activation-1.1.1.jar -aircompressor-0.10.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar -antlr4-runtime-4.7.1.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar -apache-log4j-extras-1.2.17.jar -apacheds-i18n-2.0.0-M15.jar -apacheds-kerberos-codec-2.0.0-M15.jar -api-asn1-api-1.0.0-M20.jar -api-util-1.0.0-M20.jar -arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar -audience-annotations-0.7.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar -chill-java-0.9.3.jar -chill_2.12-0.9.3.jar -commons-beanutils-1.7.0.jar -commons-beanutils-core-1.8.0.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.11.jar -commons-compress-1.8.1.jar -commons-configuration-1.6.jar -commons-crypto-1.0.0.jar -commons-dbcp-1.4.jar -commons-digester-1.8.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.8.1.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.7.1.jar -curator-framework-2.7.1.jar -curator-recipes-2.7.1.jar -datanucleus-api-jdo-3.2.6.jar -datanucleus-core-3.2.10.jar -datanucleus-rdbms-3.2.9.jar -derby-10.12.1.1.jar -flatbuffers-java-1.9.0.jar -generex-1.0.1.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-3.0.jar -guice-servlet-3.0.jar -hadoop-annotations-2.7.4.jar -hadoop-auth-2.7.4.jar -hadoop-client-2.7.4.jar -hadoop-common-2.7.4.jar -hadoop-hdfs-2.7.4.jar -hadoop-mapreduce-client-app-2.7.4.jar -hadoop-mapreduce-client-common-2.7.4.jar -hadoop-mapreduce-client-core-2.7.4.jar -hadoop-mapreduce-client-jobclient-2.7.4.jar -hadoop-mapreduce-client-shuffle-2.7.4.jar -hadoop-yarn-api-2.7.4.jar -hadoop-yarn-client-2.7.4.jar -hadoop-yarn-common-2.7.4.jar -hadoop-yarn-server-common-2.7.4.jar -hadoop-yarn-server-web-proxy-2.7.4.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar -hppc-0.7.2.jar -htrace-core-3.1.0-incubating.jar -httpclient-4.5.6.jar -httpcore-4.4.10.jar -istack-commons-runtime-3.0.8.jar -ivy-2.4.0.jar -jackson-annotations-2.9.8.jar -jackson-core-2.9.8.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.9.8.jar -jackson-dataformat-yaml-2.9.8.jar -jackson-jaxrs-1.9.13.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.9.8.jar -jackson-module-paranamer-2.9.8.jar -jackson-module-scala_2.12-2.9.8.jar -jackson-xc-1.9.13.jar -jakarta.activation-api-1.2.1.jar -jakarta.xml.bind-api-2.3.2.jar -janino-3.0.11.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar -javax.inject-1.jar -javax.inject-2.4.0-b34.jar -javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar -javolution-5.5.1.jar -jaxb-api-2.2.2.jar -jaxb-runtime-2.3.2.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar -jetty-6.1.26.jar -jetty-sslengine-6.1.26.jar -jetty-util-6.1.26.jar -jline-2.14.6.jar -joda-time-2.9.3.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json4s-ast_2.12-3.5.3.jar -json4s-core_2.12-3.5.3.jar -json4s-jackson_2.12-3.5.3.jar -json4s-scalap_2.12-3.5.3.jar -jsp-api-2.1.jar -jsr305-3.0.0.jar -jta-1.1.jar -jtransforms-2.4.0.jar -jul-to-slf4j-1.7.16.jar -kryo-shaded-4.0.2.jar -kubernetes-client-4.1.2.jar -kubernetes-model-4.1.2.jar -kubernetes-model-common-4.1.2.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.12.0.jar -log4j-1.2.17.jar -logging-interceptor-3.12.0.jar -lz4-java-1.5.0.jar -machinist_2.12-0.6.1.jar -macro-compat_2.12-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-3.1.5.jar -metrics-graphite-3.1.5.jar -metrics-json-3.1.5.jar -metrics-jvm-3.1.5.jar -minlog-1.3.0.jar -netty-3.9.9.Final.jar -netty-all-4.1.30.Final.jar -objenesis-2.5.1.jar -okhttp-3.8.1.jar -okio-1.13.0.jar -opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar -parquet-column-1.11.0.jar -parquet-common-1.11.0.jar -parquet-encoding-1.11.0.jar -parquet-format-structures-1.11.0.jar -parquet-hadoop-1.11.0.jar -parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.11.0.jar -protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar -pyrolite-4.13.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar -scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar -scala-xml_2.12-1.0.6.jar -shapeless_2.12-2.3.2.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.23.jar -snappy-0.2.jar -snappy-java-1.1.7.1.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar -stax-api-1.0-2.jar -stax-api-1.0.1.jar -stream-2.7.0.jar -stringtemplate-3.2.1.jar -super-csv-2.2.0.jar -univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar -xbean-asm7-shaded-4.12.jar -xercesImpl-2.9.1.jar -xmlenc-0.52.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.6.jar -zstd-jni-1.3.2-2.jar diff --git a/dev/deps/spark-deps-hadoop-3.2 b/dev/deps/spark-deps-hadoop-3.2 deleted file mode 100644 index 3f0a665c54ef..000000000000 --- a/dev/deps/spark-deps-hadoop-3.2 +++ /dev/null @@ -1,221 +0,0 @@ -HikariCP-java7-2.4.12.jar -JavaEWAH-0.3.2.jar -RoaringBitmap-0.5.11.jar -ST4-4.0.4.jar -accessors-smart-1.2.jar -activation-1.1.1.jar -aircompressor-0.10.jar -antlr-2.7.7.jar -antlr-runtime-3.4.jar -antlr4-runtime-4.7.1.jar -aopalliance-1.0.jar -aopalliance-repackaged-2.4.0-b34.jar -apache-log4j-extras-1.2.17.jar -arpack_combined_all-0.1.jar -arrow-format-0.12.0.jar -arrow-memory-0.12.0.jar -arrow-vector-0.12.0.jar -audience-annotations-0.5.0.jar -automaton-1.11-8.jar -avro-1.8.2.jar -avro-ipc-1.8.2.jar -avro-mapred-1.8.2-hadoop2.jar -bonecp-0.8.0.RELEASE.jar -breeze-macros_2.12-0.13.2.jar -breeze_2.12-0.13.2.jar -chill-java-0.9.3.jar -chill_2.12-0.9.3.jar -commons-beanutils-1.9.3.jar -commons-cli-1.2.jar -commons-codec-1.10.jar -commons-collections-3.2.2.jar -commons-compiler-3.0.11.jar -commons-compress-1.8.1.jar -commons-configuration2-2.1.1.jar -commons-crypto-1.0.0.jar -commons-daemon-1.0.13.jar -commons-dbcp-1.4.jar -commons-httpclient-3.1.jar -commons-io-2.4.jar -commons-lang-2.6.jar -commons-lang3-3.8.1.jar -commons-logging-1.1.3.jar -commons-math3-3.4.1.jar -commons-net-3.1.jar -commons-pool-1.5.4.jar -commons-text-1.6.jar -compress-lzf-1.0.3.jar -core-1.1.2.jar -curator-client-2.13.0.jar -curator-framework-2.13.0.jar -curator-recipes-2.13.0.jar -datanucleus-api-jdo-3.2.6.jar -datanucleus-core-3.2.10.jar -datanucleus-rdbms-3.2.9.jar -derby-10.12.1.1.jar -dnsjava-2.1.7.jar -ehcache-3.3.1.jar -flatbuffers-java-1.9.0.jar -generex-1.0.1.jar -geronimo-jcache_1.0_spec-1.0-alpha-1.jar -gson-2.2.4.jar -guava-14.0.1.jar -guice-4.0.jar -guice-servlet-4.0.jar -hadoop-annotations-3.2.0.jar -hadoop-auth-3.2.0.jar -hadoop-client-3.2.0.jar -hadoop-common-3.2.0.jar -hadoop-hdfs-client-3.2.0.jar -hadoop-mapreduce-client-common-3.2.0.jar -hadoop-mapreduce-client-core-3.2.0.jar -hadoop-mapreduce-client-jobclient-3.2.0.jar -hadoop-yarn-api-3.2.0.jar -hadoop-yarn-client-3.2.0.jar -hadoop-yarn-common-3.2.0.jar -hadoop-yarn-registry-3.2.0.jar -hadoop-yarn-server-common-3.2.0.jar -hadoop-yarn-server-web-proxy-3.2.0.jar -hk2-api-2.4.0-b34.jar -hk2-locator-2.4.0-b34.jar -hk2-utils-2.4.0-b34.jar -hppc-0.7.2.jar -htrace-core4-4.1.0-incubating.jar -httpclient-4.5.6.jar -httpcore-4.4.10.jar -istack-commons-runtime-3.0.8.jar -ivy-2.4.0.jar -jackson-annotations-2.9.8.jar -jackson-core-2.9.8.jar -jackson-core-asl-1.9.13.jar -jackson-databind-2.9.8.jar -jackson-dataformat-yaml-2.9.8.jar -jackson-jaxrs-base-2.9.5.jar -jackson-jaxrs-json-provider-2.9.5.jar -jackson-mapper-asl-1.9.13.jar -jackson-module-jaxb-annotations-2.9.8.jar -jackson-module-paranamer-2.9.8.jar -jackson-module-scala_2.12-2.9.8.jar -jakarta.activation-api-1.2.1.jar -jakarta.xml.bind-api-2.3.2.jar -janino-3.0.11.jar -javassist-3.18.1-GA.jar -javax.annotation-api-1.2.jar -javax.inject-1.jar -javax.inject-2.4.0-b34.jar -javax.servlet-api-3.1.0.jar -javax.ws.rs-api-2.0.1.jar -javolution-5.5.1.jar -jaxb-api-2.2.11.jar -jaxb-runtime-2.3.2.jar -jcip-annotations-1.0-1.jar -jcl-over-slf4j-1.7.16.jar -jdo-api-3.0.1.jar -jersey-client-2.22.2.jar -jersey-common-2.22.2.jar -jersey-container-servlet-2.22.2.jar -jersey-container-servlet-core-2.22.2.jar -jersey-guava-2.22.2.jar -jersey-media-jaxb-2.22.2.jar -jersey-server-2.22.2.jar -jetty-webapp-9.4.12.v20180830.jar -jetty-xml-9.4.12.v20180830.jar -jline-2.14.6.jar -joda-time-2.9.3.jar -jodd-core-3.5.2.jar -jpam-1.1.jar -json-smart-2.3.jar -json4s-ast_2.12-3.5.3.jar -json4s-core_2.12-3.5.3.jar -json4s-jackson_2.12-3.5.3.jar -json4s-scalap_2.12-3.5.3.jar -jsp-api-2.1.jar -jsr305-3.0.0.jar -jta-1.1.jar -jtransforms-2.4.0.jar -jul-to-slf4j-1.7.16.jar -kerb-admin-1.0.1.jar -kerb-client-1.0.1.jar -kerb-common-1.0.1.jar -kerb-core-1.0.1.jar -kerb-crypto-1.0.1.jar -kerb-identity-1.0.1.jar -kerb-server-1.0.1.jar -kerb-simplekdc-1.0.1.jar -kerb-util-1.0.1.jar -kerby-asn1-1.0.1.jar -kerby-config-1.0.1.jar -kerby-pkix-1.0.1.jar -kerby-util-1.0.1.jar -kerby-xdr-1.0.1.jar -kryo-shaded-4.0.2.jar -kubernetes-client-4.1.2.jar -kubernetes-model-4.1.2.jar -kubernetes-model-common-4.1.2.jar -leveldbjni-all-1.8.jar -libfb303-0.9.3.jar -libthrift-0.12.0.jar -log4j-1.2.17.jar -logging-interceptor-3.12.0.jar -lz4-java-1.5.0.jar -machinist_2.12-0.6.1.jar -macro-compat_2.12-1.1.1.jar -mesos-1.4.0-shaded-protobuf.jar -metrics-core-3.1.5.jar -metrics-graphite-3.1.5.jar -metrics-json-3.1.5.jar -metrics-jvm-3.1.5.jar -minlog-1.3.0.jar -mssql-jdbc-6.2.1.jre7.jar -netty-3.9.9.Final.jar -netty-all-4.1.30.Final.jar -nimbus-jose-jwt-4.41.1.jar -objenesis-2.5.1.jar -okhttp-2.7.5.jar -okhttp-3.8.1.jar -okio-1.13.0.jar -opencsv-2.3.jar -orc-core-1.5.5-nohive.jar -orc-mapreduce-1.5.5-nohive.jar -orc-shims-1.5.5.jar -oro-2.0.8.jar -osgi-resource-locator-1.0.1.jar -paranamer-2.8.jar -parquet-column-1.11.0.jar -parquet-common-1.11.0.jar -parquet-encoding-1.11.0.jar -parquet-format-structures-1.11.0.jar -parquet-hadoop-1.11.0.jar -parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.11.0.jar -protobuf-java-2.5.0.jar -py4j-0.10.8.1.jar -pyrolite-4.13.jar -re2j-1.1.jar -scala-compiler-2.12.8.jar -scala-library-2.12.8.jar -scala-parser-combinators_2.12-1.1.0.jar -scala-reflect-2.12.8.jar -scala-xml_2.12-1.0.6.jar -shapeless_2.12-2.3.2.jar -slf4j-api-1.7.16.jar -slf4j-log4j12-1.7.16.jar -snakeyaml-1.23.jar -snappy-0.2.jar -snappy-java-1.1.7.1.jar -spire-macros_2.12-0.13.0.jar -spire_2.12-0.13.0.jar -stax-api-1.0.1.jar -stax2-api-3.1.4.jar -stream-2.7.0.jar -stringtemplate-3.2.1.jar -super-csv-2.2.0.jar -token-provider-1.0.1.jar -univocity-parsers-2.7.3.jar -validation-api-1.1.0.Final.jar -woodstox-core-5.0.3.jar -xbean-asm7-shaded-4.12.jar -xz-1.5.jar -zjsonpatch-0.3.0.jar -zookeeper-3.4.13.jar -zstd-jni-1.3.2-2.jar diff --git a/dev/run-tests.py b/dev/run-tests.py index 7aa93dd95c11..f904a397f83c 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -342,14 +342,9 @@ def get_hive_profiles(hive_version): def build_spark_maven(extra_profiles): # Enable all of the profiles for the build: -<<<<<<< HEAD - build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags + build_profiles = extra_profiles + modules.root.build_profile_flags mvn_goals = ["dependency:purge-local-repository", "-Dinclude=org.apache.parquet", "clean", "package", "-DskipTests"] -======= - build_profiles = extra_profiles + modules.root.build_profile_flags - mvn_goals = ["clean", "package", "-DskipTests"] ->>>>>>> upstream/master profiles_and_goals = build_profiles + mvn_goals print("[info] Building Spark using Maven with these arguments: ", " ".join(profiles_and_goals)) diff --git a/pom.xml b/pom.xml index 8ac3207ba28f..3bd5ef74a933 100644 --- a/pom.xml +++ b/pom.xml @@ -133,18 +133,6 @@ 2.3 -<<<<<<< HEAD - 2.2.0 - 10.12.1.1 - 1.11.0 - 1.5.5 - nohive - com.twitter - 1.6.0 - 9.4.12.v20180830 - 3.1.0 - 0.9.3 -======= 2.6.0 10.14.2.0 @@ -153,7 +141,6 @@ 9.4.36.v20210114 4.0.3 0.9.5 ->>>>>>> upstream/master 2.4.0 2.0.8