From 6906464181b145854b62de744cf1cd362fd48d07 Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Wed, 16 Nov 2016 17:13:07 +0000 Subject: [PATCH 1/5] write timestamps as long since parquet has timestamp_micros --- .../SpecificParquetRecordReaderBase.java | 2 +- .../datasources/parquet/ParquetFilters.scala | 30 +++++++++++++++++++ .../parquet/ParquetRowConverter.scala | 4 ++- .../parquet/ParquetSchemaConverter.scala | 9 ++---- .../parquet/ParquetWriteSupport.scala | 19 ++---------- 5 files changed, 39 insertions(+), 25 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java index 0e46e062c94bb..e667473b05076 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java @@ -140,7 +140,7 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont + " out of: " + Arrays.toString(foundRowGroupOffsets) + " in range " + split.getStart() + ", " + split.getEnd()); } - this.reader = new ParquetFileReader(configuration, file, footer); + this.reader = ParquetFileReader.open(configuration, file, footer); } this.fileSchema = footer.getFileMetaData().getSchema(); Map fileMetadata = footer.getFileMetaData().getKeyValueMetaData(); 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 28a544c67ba3c..342c54410bfd3 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 @@ -49,6 +49,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.eq( binaryColumn(n), Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.eq(intColumn(n), v.asInstanceOf[Integer]) } private val makeNotEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -70,6 +75,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.notEq(intColumn(n), v.asInstanceOf[Integer]) } private val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -88,6 +98,11 @@ private[parquet] object ParquetFilters { case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.lt(intColumn(n), v.asInstanceOf[Integer]) } private val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -106,6 +121,11 @@ private[parquet] object ParquetFilters { case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.ltEq(intColumn(n), v.asInstanceOf[Integer]) } private val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -124,6 +144,11 @@ private[parquet] object ParquetFilters { case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.gt(intColumn(n), v.asInstanceOf[Integer]) } private val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -142,6 +167,11 @@ private[parquet] object ParquetFilters { case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) + case TimestampType => + (n: String, v: Any) => FilterApi.eq( + longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) + case DateType => + (n: String, v: Any) => FilterApi.gtEq(intColumn(n), v.asInstanceOf[Integer]) } /** 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 33dcf2f3fd167..fec62e2e90617 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 @@ -252,8 +252,10 @@ private[parquet] class ParquetRowConverter( case StringType => new ParquetStringConverter(updater) + case _: TimestampType if parquetType.asPrimitiveType().getPrimitiveTypeName == INT64 => + new ParquetPrimitiveConverter(updater) + case TimestampType => - // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. new ParquetPrimitiveConverter(updater) { // Converts nanosecond timestamps stored as INT96 override def addBinary(value: Binary): Unit = { 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 b4f36ce3752c0..a7214d8af7134 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 @@ -159,6 +159,7 @@ private[parquet] class ParquetSchemaConverter( case DECIMAL => makeDecimalType(Decimal.MAX_LONG_DIGITS) case UINT_64 => typeNotSupported() case TIMESTAMP_MILLIS => typeNotImplemented() + case TIMESTAMP_MICROS => TimestampType case _ => illegalType() } @@ -368,14 +369,8 @@ private[parquet] class ParquetSchemaConverter( // from Spark 1.5.0, we resort to a timestamp type with 100 ns precision so that we can store // a timestamp into a `Long`. This design decision is subject to change though, for example, // we may resort to microsecond precision in the future. - // - // For Parquet, we plan to write all `TimestampType` value as `TIMESTAMP_MICROS`, but it's - // currently not implemented yet because parquet-mr 1.7.0 (the version we're currently using) - // hasn't implemented `TIMESTAMP_MICROS` yet. - // - // TODO Converts `TIMESTAMP_MICROS` once parquet-mr implements that. case TimestampType => - Types.primitive(INT96, repetition).named(field.name) + Types.primitive(INT64, repetition).as(TIMESTAMP_MICROS).named(field.name) case BinaryType => Types.primitive(BINARY, repetition).named(field.name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index a31d2b9c37e9d..6f7f7aef79f47 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.nio.{ByteBuffer, ByteOrder} import java.util import scala.collection.JavaConverters.mapAsJavaMapConverter @@ -32,11 +31,11 @@ import org.apache.parquet.io.api.{Binary, RecordConsumer} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter.minBytesForPrecision import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ + /** * A Parquet [[WriteSupport]] implementation that writes Catalyst [[InternalRow]]s as Parquet * messages. This class can write Parquet data in two modes: @@ -154,20 +153,8 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit Binary.fromReusedByteArray(row.getUTF8String(ordinal).getBytes)) case TimestampType => - (row: SpecializedGetters, ordinal: Int) => { - // TODO Writes `TimestampType` values as `TIMESTAMP_MICROS` once parquet-mr implements it - // Currently we only support timestamps stored as INT96, which is compatible with Hive - // and Impala. However, INT96 is to be deprecated. We plan to support `TIMESTAMP_MICROS` - // defined in the parquet-format spec. But up until writing, the most recent parquet-mr - // version (1.8.1) hasn't implemented it yet. - - // NOTE: Starting from Spark 1.5, Spark SQL `TimestampType` only has microsecond - // precision. Nanosecond parts of timestamp values read from INT96 are simply stripped. - val (julianDay, timeOfDayNanos) = DateTimeUtils.toJulianDay(row.getLong(ordinal)) - val buf = ByteBuffer.wrap(timestampBuffer) - buf.order(ByteOrder.LITTLE_ENDIAN).putLong(timeOfDayNanos).putInt(julianDay) - recordConsumer.addBinary(Binary.fromReusedByteArray(timestampBuffer)) - } + (row: SpecializedGetters, ordinal: Int) => + recordConsumer.addLong(row.getLong(ordinal)) case BinaryType => (row: SpecializedGetters, ordinal: Int) => From 6d599a7b8984447f477bcef04d349a7fe8ccaf5b Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Wed, 30 Nov 2016 19:28:11 +0000 Subject: [PATCH 2/5] types --- .../datasources/parquet/ParquetFilters.scala | 25 +++++++++++-------- .../parquet/ParquetWriteSupport.scala | 1 - 2 files changed, 15 insertions(+), 11 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 342c54410bfd3..8bc53c60bfddb 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 @@ -76,10 +76,11 @@ private[parquet] object ParquetFilters { binaryColumn(n), Option(v).map(b => Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])).orNull) case TimestampType => - (n: String, v: Any) => FilterApi.eq( + (n: String, v: Any) => FilterApi.notEq( longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) case DateType => - (n: String, v: Any) => FilterApi.notEq(intColumn(n), v.asInstanceOf[Integer]) + (n: String, v: Any) => FilterApi.notEq( + intColumn(n), v.asInstanceOf[java.sql.Date].asInstanceOf[Integer]) } private val makeLt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -99,10 +100,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) case TimestampType => - (n: String, v: Any) => FilterApi.eq( + (n: String, v: Any) => FilterApi.lt( longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) case DateType => - (n: String, v: Any) => FilterApi.lt(intColumn(n), v.asInstanceOf[Integer]) + (n: String, v: Any) => FilterApi.lt( + intColumn(n), v.asInstanceOf[java.sql.Date].asInstanceOf[Integer]) } private val makeLtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -122,10 +124,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) case TimestampType => - (n: String, v: Any) => FilterApi.eq( + (n: String, v: Any) => FilterApi.ltEq( longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) case DateType => - (n: String, v: Any) => FilterApi.ltEq(intColumn(n), v.asInstanceOf[Integer]) + (n: String, v: Any) => FilterApi.ltEq( + intColumn(n), v.asInstanceOf[java.sql.Date].asInstanceOf[Integer]) } private val makeGt: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -145,10 +148,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) case TimestampType => - (n: String, v: Any) => FilterApi.eq( + (n: String, v: Any) => FilterApi.gt( longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) case DateType => - (n: String, v: Any) => FilterApi.gt(intColumn(n), v.asInstanceOf[Integer]) + (n: String, v: Any) => FilterApi.gt( + intColumn(n), v.asInstanceOf[java.sql.Date].asInstanceOf[Integer]) } private val makeGtEq: PartialFunction[DataType, (String, Any) => FilterPredicate] = { @@ -168,10 +172,11 @@ private[parquet] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromReusedByteArray(v.asInstanceOf[Array[Byte]])) case TimestampType => - (n: String, v: Any) => FilterApi.eq( + (n: String, v: Any) => FilterApi.gtEq( longColumn(n), v.asInstanceOf[java.sql.Timestamp].asInstanceOf[java.lang.Long]) case DateType => - (n: String, v: Any) => FilterApi.gtEq(intColumn(n), v.asInstanceOf[Integer]) + (n: String, v: Any) => FilterApi.gtEq( + intColumn(n), v.asInstanceOf[java.sql.Date].asInstanceOf[Integer]) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 6f7f7aef79f47..5a08973078e20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaConverter import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ - /** * A Parquet [[WriteSupport]] implementation that writes Catalyst [[InternalRow]]s as Parquet * messages. This class can write Parquet data in two modes: From 65b7f2d59dbe17a0c8689741655ff385501505d9 Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Wed, 30 Nov 2016 19:40:30 +0000 Subject: [PATCH 3/5] don't pushdown if int96 is timestamp --- .../parquet/ParquetFileFormat.scala | 16 +++---- .../datasources/parquet/ParquetFilters.scala | 47 +++++++++++-------- 2 files changed, 35 insertions(+), 28 deletions(-) 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 173ee97ae4a9f..ed3ea5be3fc76 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 @@ -21,10 +21,6 @@ import java.io.FileNotFoundException import java.net.URI import java.util.concurrent.{Callable, TimeUnit} -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.util.{Failure, Try} - import com.google.common.cache.{Cache, CacheBuilder, RemovalListener, RemovalNotification} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -39,8 +35,6 @@ import org.apache.parquet.hadoop.codec.CodecConfig import org.apache.parquet.hadoop.metadata.ParquetMetadata import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType - -import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -52,6 +46,11 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.{SparkException, TaskContext} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.{Failure, Try} class ParquetFileFormat extends FileFormat @@ -382,13 +381,14 @@ class ParquetFileFormat requiredSchema).asInstanceOf[StructType] ParquetWriteSupport.setSchema(dataSchemaToWrite, hadoopConf) + val int96AsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp // Sets flags for `CatalystSchemaConverter` hadoopConf.setBoolean( SQLConf.PARQUET_BINARY_AS_STRING.key, sparkSession.sessionState.conf.isParquetBinaryAsString) hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sparkSession.sessionState.conf.isParquetINT96AsTimestamp) + int96AsTimestamp) // Try to push down filters when filter push-down is enabled. val pushed = @@ -397,7 +397,7 @@ class ParquetFileFormat // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` // is used here. - .flatMap(ParquetFilters.createFilter(requiredSchema, _)) + .flatMap(ParquetFilters.createFilter(requiredSchema, _, int96AsTimestamp)) .reduceOption(FilterApi.and) } else { None 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 8bc53c60bfddb..5562176c0feed 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 @@ -17,10 +17,9 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.parquet.filter2.predicate._ import org.apache.parquet.filter2.predicate.FilterApi._ +import org.apache.parquet.filter2.predicate._ import org.apache.parquet.io.api.Binary - import org.apache.spark.sql.sources import org.apache.spark.sql.types._ @@ -188,23 +187,29 @@ private[parquet] object ParquetFilters { * using such fields, otherwise Parquet library will throw exception (PARQUET-389). * Here we filter out such fields. */ - private def getFieldMap(dataType: DataType): Map[String, DataType] = dataType match { - case StructType(fields) => - // Here we don't flatten the fields in the nested schema but just look up through - // root fields. Currently, accessing to nested fields does not push down filters - // and it does not support to create filters for them. - fields.filter { f => - !f.metadata.contains(StructType.metadataKeyForOptionalField) || - !f.metadata.getBoolean(StructType.metadataKeyForOptionalField) - }.map(f => f.name -> f.dataType).toMap - case _ => Map.empty[String, DataType] - } + private def getFieldMap(dataType: DataType, int96AsTimestamp: Boolean): Map[String, DataType] = + dataType match { + case StructType(fields) => + // Here we don't flatten the fields in the nested schema but just look up through + // root fields. Currently, accessing to nested fields does not push down filters + // and it does not support to create filters for them. + fields.filter { f => + !int96AsTimestamp || + f.dataType != DataTypes.TimestampType || + !f.metadata.contains(StructType.metadataKeyForOptionalField) || + !f.metadata.getBoolean(StructType.metadataKeyForOptionalField) + }.map(f => f.name -> f.dataType).toMap + case _ => Map.empty[String, DataType] + } /** * Converts data sources filters to Parquet filter predicates. */ - def createFilter(schema: StructType, predicate: sources.Filter): Option[FilterPredicate] = { - val dataTypeOf = getFieldMap(schema) + def createFilter( + schema: StructType, + predicate: sources.Filter, + int96AsTimestamp: Boolean): Option[FilterPredicate] = { + val dataTypeOf = getFieldMap(schema, int96AsTimestamp) // NOTE: // @@ -256,18 +261,20 @@ private[parquet] object ParquetFilters { // Pushing one side of AND down is only safe to do at the top level. // You can see ParquetRelation's initializeLocalJobFunc method as an example. for { - lhsFilter <- createFilter(schema, lhs) - rhsFilter <- createFilter(schema, rhs) + lhsFilter <- createFilter(schema, lhs, int96AsTimestamp) + rhsFilter <- createFilter(schema, rhs, int96AsTimestamp) } yield FilterApi.and(lhsFilter, rhsFilter) case sources.Or(lhs, rhs) => for { - lhsFilter <- createFilter(schema, lhs) - rhsFilter <- createFilter(schema, rhs) + lhsFilter <- createFilter(schema, lhs, int96AsTimestamp) + rhsFilter <- createFilter(schema, rhs, int96AsTimestamp) } yield FilterApi.or(lhsFilter, rhsFilter) case sources.Not(pred) => - createFilter(schema, pred).map(FilterApi.not).map(LogicalInverseRewriter.rewrite) + createFilter(schema, pred, int96AsTimestamp) + .map(FilterApi.not) + .map(LogicalInverseRewriter.rewrite) case sources.In(name, values) if dataTypeOf.contains(name) => val eq = makeEq.lift(dataTypeOf(name)) From d885cac4a3fb5623955da8edcc340426040e7de0 Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Wed, 30 Nov 2016 19:41:51 +0000 Subject: [PATCH 4/5] imports --- .../datasources/parquet/ParquetFileFormat.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 ed3ea5be3fc76..922c20fa71f73 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 @@ -21,6 +21,10 @@ import java.io.FileNotFoundException import java.net.URI import java.util.concurrent.{Callable, TimeUnit} +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.{Failure, Try} + import com.google.common.cache.{Cache, CacheBuilder, RemovalListener, RemovalNotification} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} @@ -35,6 +39,8 @@ import org.apache.parquet.hadoop.codec.CodecConfig import org.apache.parquet.hadoop.metadata.ParquetMetadata import org.apache.parquet.hadoop.util.ContextUtil import org.apache.parquet.schema.MessageType + +import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -46,11 +52,6 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration -import org.apache.spark.{SparkException, TaskContext} - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.util.{Failure, Try} class ParquetFileFormat extends FileFormat From f19c8a0415b82731b43b1a0ee66f047bdc49a22d Mon Sep 17 00:00:00 2001 From: Robert Kruszewski Date: Thu, 1 Dec 2016 14:34:00 +0000 Subject: [PATCH 5/5] types --- assembly/20ts/._SUCCESS.crc | Bin 0 -> 8 bytes ...-4627-9227-5ac0947a4b18.snappy.parquet.crc | Bin 0 -> 636 bytes assembly/20ts/_SUCCESS | 0 ...522c-4627-9227-5ac0947a4b18.snappy.parquet | Bin 0 -> 80335 bytes .../parquet/ParquetFileFormat.scala | 2 +- .../parquet/ParquetFileSplitter.scala | 32 +++++++++++------- .../datasources/parquet/ParquetFilters.scala | 3 +- 7 files changed, 22 insertions(+), 15 deletions(-) create mode 100644 assembly/20ts/._SUCCESS.crc create mode 100644 assembly/20ts/.part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet.crc create mode 100644 assembly/20ts/_SUCCESS create mode 100644 assembly/20ts/part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet diff --git a/assembly/20ts/._SUCCESS.crc b/assembly/20ts/._SUCCESS.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/assembly/20ts/.part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet.crc b/assembly/20ts/.part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..2cb31be7432c8b610514554cf23bb235acf42d33 GIT binary patch literal 636 zcmV-?0)zcya$^7h00IDleJFEB7?D`EQPGQS;9-E*&hBBY=&q_B`*Xo+P`rh~LJy^~ zTC=SjR+!o3O)l#)aJ-e1^zJ(UF>Si)K#oaUb+y^w5R3hByyp+hwmZGZ*PLK2%ADk7 zW997K`h2^f>j*}ugWKB%Ue+rwf$B4s`KR|-gr=!GB21ke_fB$U3Uk%l&SN5l%=6+S zMD~`$*4nGIt}$~)UxKKQt%uV?iO?;b9+gx+P4nWZ%WCk%4SQUwMU<5Fb)lk3n1I#< zwOLV_0!01lCD@5UVh7ENk3y+=8fsSPuRbfI<+f#+ z$?BS1ELe;Q^`)GPet??d(vf)^4qOgmL8OZGz+~zf>ew`@IF=;A9Ve(W17|O8*T#1~ z4&Zb#DLZr&MoXLycx$0d1&L_sAEi|YD4{5zyEn`%d`o5q!WylqUUw83wE_z$ z!LIdYx&h`%F|eD1frkg60<`()>8nauYlwg^wRflW@LFwv9|OWHW##J4^d&70$krqE zZASWYm&GSzoHha?Oo>cwLx7Y7x}Zf^h)fAw(-1+@VE~kEyUU;C2%7zs72Oy_w#W6 WU0AeBFR7M^d#=#(%2amqZs!z3FghIo literal 0 HcmV?d00001 diff --git a/assembly/20ts/_SUCCESS b/assembly/20ts/_SUCCESS new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/assembly/20ts/part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet b/assembly/20ts/part-00000-00140901-522c-4627-9227-5ac0947a4b18.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..3cf5e170f43ee12a43702de908053fb46d07b16c GIT binary patch literal 80335 zcmWifVK|p}A4iiUNs=VVB$G^%WRfIFl1!3hl1Y*z$s|dVBuOSol1Y*zNs=TsH~jyJE8NvEh;Z88U|0TBv=%VbR@{G zS&VA>6a>>F3!u;RKC)rS>~(xeY~5B=^_O&*5YtLGFq(S~P) zD&Gk%g!r*5Q7##gP;*H+q3`AJ4TR!=g{X)L!i=`rc~EJ254ru#Ojo{?6TBG}I9Zs{ zEU%d$y;5-_cMb2?pi;w*L9eO8jQWKVZUm7vW~jeVrGpD(8r&#WUYI8||**~5V0^-MQCYa7yfs(2UhxP8IFlD$fqi5254<6K; zw-Yt=BnMV_brE7mX>8%nDRm=Cec>6{^Q@6j=qv9@2pcQRXq|BsDlB!+ zg7Tjr%xIE*14=IcAGz+0wijPY4)#QOOcrL;&3jE4wEk%ucjfOLP%&X~&~d6Tqgvq@ zZ$h`-GE~8*P#79n46UYrC(D=ox}67G9Cx5Hzhptbm=1#ROf?_wB+CCl)w-O9Nk8fd z?(woa2s(4DQQsdPhGp*e3DLuc`w|?Ij8Llc0$|sZYcSvI6FFqm%s=^(WvV+$Zec2H ze)f`(;XB!nJKwRwj7AwTP<%-hq2}eNorJ`IB`CKE!i?J4g>b;4pIrK8jz14Z1#d&y zPZnlW$!jBYTC42hF7N$1RB+fyXfaioQMOPzfY4-TiAw);5c)*kg@)7rBa4+7?B>C0 zM?X};mn+aU<^w@%=8Qn@`tQ4z`%E7bieF0Z zBSZw4qiiP#Gb&}@hV2&L$T@HHLU=GR*au}cS(s5e?;WAR`se-JrM`DYd4(lH{i(u? zB8B5a36*xgq2fM8KIviW<@nh80h4!dR~_WT#Pc4)LIRsu!xq zC<7KgdqW8Gof^TN^;ls>g^YOEYFbOkdO7wm!9QR*%4C8tqhxjotg{#-C%>5&$%7uj zf1-3J3o{O0>m-z0s~_Pm=KThgW7ywNZK^P1_f5GdLV=wPD)iG4XchSnL4Nv(qlA__ z#;8okK?OqQ5AE1ECW-uFNyxuik&AFl{H@l*cd?z_%fRN2E=80}t7a2PHb zLr_gxit5te5A&ZC!4R(jvgIhfV>~FAx*gSQbO~lWYbW^ns>X6>G**~VJmVCsG5wd2 z_)_LL!7acFr9DBIap3YjLaD{DI6~ALLzI26KUA44%-DJDBRS9dmlJ#`_`Ms-A}j^U zP8DWsx;ZJHyL7wND4$Q!&@i%+AU0k6B%%7wVpM|TUg-Lz0BXhbk^41u5_quW{#I0^ z%LQonqm`f>uY8KT4jo5S?!yx>$o&byeE8Tzf=rSHs!=}-raid>y}f>r4Mxp7&4Z$; zzNjjrY#9ISJ;B9ydJ=bwqGtM@g2x*Zlx}bk98?fyEWh?Ya*Xxti+suPy(da7 zEFE?$3o{nn{3(;WP`lqzR-fXad}IxwMNR4wA@j~Ml%L}PX#AxZO2m98*J|o#@nF*Z z9VmB~%TVV>2jTk(waeU<>A0YxAD)H|?#~FS!zW}Dx=vf8^7SKO$dmg7ORwQq2y&x@ z8Jo`rzzm~2==WcEB3cfUGcM>E$Xyl!Zht z!T!w>luGbE*r_1Qn0Kw89Bi$7jR!5>Z$rt3U4%`_!i?!RRr0v=v2#KhemV)oBA*hf z)y7>XB;2t?xjKeJtuJ?B|6l)+OEl-_^I+tCKa`zI4pjc}fzWYc#trUrbvB}c9wtL` z_eO%uaQOm4ci z&cg`P*96;_O1HRE3UENR%L+5*Tz*Ulv=~!FFneQ;k`4}q4GO}Hsn@=dy{vz|&4c>y zeNZA{S+G)Bm@)3=&&Aw1+x>yk_>>5H57!fl)MW1v!teZsvT-~N6~B}b+WrzLA!KVB zp#tvjf~GFlpwy3Ua{UR-yL>4{#~tPQ@GR7Ge@Pe;nOw?U#c4ZKtbPo1dh(E6@2HE@nEZzFk{w*4Crs#Mlg9f^*(o!0qan8vcinXmum?g z7SiPe-8aiogL@Ccas^?=m}`S%M{9$Bcu?*ApQ!EwS73p%Fk|RV^$PB+>|9aupZUYX2HhNl4OMgmS+h2z6ZY;rH*K$z>;IJ>*N#I$KZ< z57VHkdkdjUM5&6q{L^buA^OLm<&$cHoR{PyLbJFqW5(J2(AVfTG~cnx$_g_^UH+F~Z!x}xpz_8F)wwqU<|zm>245TYgkWL4 z044k0A2uDxh3U$|j6OGispU(CcAHRQpHg7;;U+?Yn!;1=T<@$#X*vD{`}-;hC4Y(6 z5h68BQFizDLS>hmu;Y6lIroIlzdRVEw84V{%dqIAwY8}%JhB+lyWJ8_20jeQ%=l%#e<$Y z+fjNCGvUytc0z@SY72Lvd~Y<2Q4nTyyf*SJ zLCtzGs=Ffy791#mp~}LHRySw2^Cfw^t*DmnbeMU#mEfnQ{Ej>0JHMkO98bX7z8XT( zUs4?ecTEeF&iw=M`yY2;+4t|{=o9+ydC)<}7p3~}GVI#)o{%pxy_36;(=I4W{Uj*& zx2)43~^yb-?Nd>$f%qk{%m*`p=QTIRHBqHquYf%sBPLq9%!2RkuR0*+Kh^l z6=t-*+)PlhQ0(TevvmzBZ|^Y}tRT#2aZTbALDt#~)zq;MrXRQkeUyb64R7l9@SvET zH>$e(B1|~^mf)&3?K5{;cbriDzbC+wzNdu9zsB_v>@-)Pl<$Ybjz3Ca?)U%5K_}+_ zj|a_lcA{h+=D@~HU4%3djXv(YPj5sS=%0b2PZ|kTTjjqH;>Cp-UCthbnnn)@ef47f zgyJ10s0b-xM%xQFppxnT$n8zqU-?qbE>BdTtT3b5<<|shi=PI#YiM;qrS6S`UJAmD z`q##MBZycpLsfQ!!ngy)&{wQJHN2;IL~q6#_=!q5YE zp_Q^Qr2I_-F&=DL?T57R$qH7$gUtxBW0^A#6{ zG&*}6iW@y9)YMB#5)yZqquiu~A+;~uh69VgkxQHO#_(X&E+3S=tT3d?<#&Y6Wj{-C zm)Giy3f`LtEfjSw^Jnm2^_ty3p(=OQl5sChOX_ARI-6xWZHBSh?2jmUR z0^1i4l5?8oP2|DAU4Np?WQ7@}FLx3emZ{5gm)g1k<+b;3sIMT*C~{415~1=p8&q7! z5$Js2AA*MR2n9lKfibFRbs!Ayz7A~;e=Y+KN3;*}td3fWB7*)4!%Dawny^ z7FAys3se56CU|}yJ%ylmVkv59_I_AVSp;J@4UnBg^pto|{q%NJ&##wYVNE+BY^&-| z+*ykYGb)@t1zQ*VOUSC1Q6~8ButJ$g2{TGwxJRg4JZvfFU7RFp&a+7Ks5zn#_rrn)3_`6Z8a*iBN|#As3gcMi>ne^3KpX>SMP;>-38G2 za35JhP3LDGtS#P(O8Wf*bnj~===`Ou#@+W{98qOuCt&m+PY4d*rKS^9PgtP3W{1K2 z$~!P*(+{$x$UJo(lsoNs(gF7Q}VMg(@Nw8)?10k_~f(F5D$4Zp8 zlrZB!dO4wVvFI;^sHTM|`(3-CimWhWXI2k6Z<&@RUkYyBjI!8!4$3MBGdATa&g3rr zH+z&%$1!Ml;1NMgd9)Uxy1)#TusQ^~cHe?phrg2hf1W#w2TO{*QIWrALc6}V1m(Y` zX>-@{%X(C9Spp3DqmE$yecWt<%!w7K#@UBpT4gEp-ZVru5HZx@LDADYQB}WQh4D3A z1edKEbGXwK7iR2BIRlFqydXr>%j*(sci5toq=XsU(;pCW7LS}u2y8Mzne7UK(z3#g z4O#yqr!JeV$CtcXJyH65)1iohFk@xzPrq^(_uKC%=Z-k2aiE6K`;(MDp{QUPDtz?; zXwzK`6%T(WxBaX?j|a1hcc21(zYIezFmF zCOg)kB&CEI>(Z+T$%`cx5Ntm%9 zS7i}*p}#qytU69Y`2$Z0EkBJjA!HUSJX z?*rj`^bAw(%6{31iY`lr4u3QfRKHJLLg3W2DyZx%H2>ozLFT){ zZ`?J;*`d;A$3XAOhXjL7qpS&{BEpPSiNP@b*Fxx0^M$Orb>DY}Se%jpBNns~ zZ0o1maHq6m9jbkdFk?=7Eg^8R^eTc`({hybu7j{)yf9;G)*#tynSm`2>bL%h64`qN zR!$OTjLTJD&7Jdat|*O;zhQ6ab3)Ni6YU7$1vV&~)lpEf`yWDE#E3P7?4K8*0*V8n z>F@bas_!$oK6;itUrPC93(B)B4eI^TLKylsWi59VacfbrvyVfk%4&l8CP@cE&oE)e z!o>YB?AP1Sx@Lf^uvPDO9&8;c%$SvO3HmR1M=+^Zb>vQRhcl{fj4)$z`o9E^#pBlz zbepVDgZ>e)e7rDYOx7?bg5$CUD7999*d3G$3nmFOhUWgVo-bMbwh1NQkpf#nn+Tad zDL8ZISFjpoy!tOF(OpTXjS&BXko2=D%Ds3m)cO4;{Qkc_a#^&F3lB#B;)QZ3y8u=H zcthy=ro4f>{5VHc$n1D%S^0z@w`r^^p?R1vV@6^a^!>F28rA$Di*KE`kq2u=3Nt3A zWJ9+FodoUr>2BN&_-sIxjuB>zN^c<8FP7UxP-$9;>hwPX^TrD^24{)76D*b)qhwom z!=|9?Fny9Rqff5ZX1-+jn+Hm);~cCGeMLz4X^IDTt_Aiet<|xx|KlSxRM?0bM+qA9RrNymBdCxut4JzvhqMKyA z2vx&`8RHWVL6=|eLCu;Wa-XN+HXbY}mjLx~Uckm^R-?pN9-=)K% z&{jhDPs+aB*%bVaQe1rkwtcK2WJgH-NeKAa0%cl!080IS2iE`ZJ2@p<-;W19fAK}> zm0gBI&hH5o-_&+;7aQk-a+;k4)hnM7dfX=X6AFh3GlnHbLhE142?{mCcM)1W7oxI8 z3N!krj{w&vWEzEKU<@eix0yNhx>%w|A|Bpf})L3 z=D!3$nX+rJ(fJcO?VILdzT_R}jxv~?3Pmel5~|!LM{*ZGOqkInF$QY>T1Du4Jn9Ic z*mDUgVx%ymZAu|jTF_5!|94Il59au6Lj{fzW;9E0BS2`#Birr;EHOqO@`SY zpA!NiCLSZ0{=5n$RU8HD9V!SZ{~HlY@QhxB()(o(9J-eeE1Y}f#!QI)HtYC;VH3xh zjB<+Gf>ob=9`-zF;k3|ADUQ>yVZxHuiN~SBua60>k0nnKvOLXE{v(AYO;T<{$pzoY zb^q$c^QTYt@j-cv5th_Ve@7TJ`S~Pws&z~s!F~X86>79g56ZNy) z<-OQ|3ikgST8tN#&a&SjV|qh{hM*r)d|q4=@PMM8w970Py`Fr!k+ zJwm(TuuMYEzlNwlAAe{zMwn4L{Uf=-W>#@6v>*D#a*TOYE<0o zXy_bNNzj-iewomFbup^QYA+0bR{(87`^btv>16X@+l{TLY}*Sk;A1PnG(!0bcTzt) zqUvv-fGG}72%i6w$|2}QTcCzC!eGU{J22My2ifVH{#71SkMl+KXlKL12k!}CZqswQ zvmPeQsF0WhTlE?US&t`NBlvr+M45~fW|T}RC)62=<`I(rU5N7V*$s8a2r~|z?;)3) zXkF(^F)ucw9R1HhweiA?-Io;exhpWYM}@vV2Cae~5#%S0zCmcYYKF?R3W0v_Zb9SF zuVjgz<`(c^?G0~Kl5Hk*|M-@m6EW>3ci&alqsne4z-Wg$g2VsD6%tgVSD?By4#E6; zr7*;Kh-~?7{w*Gqi`$86*1ife9&{0W-8720Ga4q$D4uu**66(;BtDkEO>pzHMQM)| zW*j*4fKX~UvX~I{uL;WDCkU#H5oYW>|37k`$?Q9PDfopa%ECV#%8nOiY`XMQ33ut{ zzoUF!$3erO8iLp)sk?;gtIJRcRtKQ#yJDyn`kmabq+iN|B{z1UB5f~2yN?|N>GZl0NWgB`tbunDq_YX?&?)HqEc=rLr;fBg5LioRuhJfu0mC4 z9EGv>DhN)_Vvh;x-xi^Ij_-km+Baa>gI=<=n|2LfQWz%8*m~+b%+h;J@PDlIggX;Y z2bAPUVaB>Mj|s_!V`>Q=|C*z8eL~@&q%dRo`ETSHlV6|mpyLZ4l$w7Q?3NK`EV%S@ z9e1JTf1s>hCqnt4dP0l5?7xJ}tG}WAtPVrtcVz^LP?35)@E(7;{SQ9D$=MhpzO_mE)eJnA_??wcv9`FJqQ&@P0& z55AC%+~zd$p!hIh#+p+ZFj23K;P!ax3+}W%*P#Z)gc(cE)Dof$rJD%$|1L+V_#A|t zlERF6=LgBbCI&Bg(Bj3PC|Un2ut`RkG5wNyGj~4bt|-ITe?zgL=Y(qciLVF=S8Y(P zR#8yv-9Lo>{UcfkB}&4Kkv9UNoozl;{`i^PacI_SzLcxF1r>BV4VpW&5M=&0rIowJ zqia!V8pomcy=sDiv*a6s=(nY)s^j}%y!LJA@?e0h>897lgMCBWQN^b&!3e!~1lz}| zZ@E+QbVjv{2{Y!L`IiuAIKG`=_OBI6+9v`wND4Eio*(v(;AOG^rT@YoiumWkN*Q6s zxJ$ou@Fi#SO(>1mDX@1>6QM|6;XQZZS68EKtp0+E?SG_d{*cZG9!yd7LV4c40QDT+5Qch{ySS@3>WGTfh=)%1o)FZX$9^RA3|OEFkB7l9 z?GkAH;0Ia3ZC*DIwhsBCvQA|~f4xqE$>ZstxRdnUfT|M{W=uZQK=3e>>mlg=yAn0H z{Rk|V6lRP$FZ!9_Xkv^~d$AjK@4OBRWP}+*FKPAiB`b3el>F;+uw~CHLZTaJtEZZAKgbtQW9o#zYzj;Y>VLcuCL^>Lvz3IV6>_*qr>e?sOr#8=<1!; z&t3k}^{5byQ_%8W9YM}n<}0CjU!*A~^eJsRdo3Nr?uANhk|VX_z{`yvQ7?JR)l zGQx~Lmu3(0B}4PAD6!Y+uzF7`AwgdGKki(w{*KbJIsyCJYX~L#rG^NRO2UkGHx58$ z+dHtM>pOp)&OM|*%xKueJ949fRD~tYZ(oKo4(~Z_>{S!tH0`Jh%3C7|8r*wE5Otm~ zoKQ7jjfy`W30<_y37QXvixT=aE<_a%1;B_?dC*p`hphB?<_P}u?OQgZa>Rrs1J5)Q z%nTL9xRd^O4XR=LF_4wzstQZ$ z-OhnSYr6;)y)z`ai#@s#<)m>2s^4oQ^sJX3Lns_rg$g@<6k2OPASgT#lOnWkG(lwz z?ScNMZa@>g|B)phYmenibz3}9$zsBi9%o(?bPa!!=5Dag0ad;|4#r3dOFEt(GmfBU zvJBPT7zzt^7Q;{(mK4I1In9x=$$U5}81VK4gS)P!(oWy?qUKt^GvK@70v!OCd+yQI;C1Q10GK zLi75`6S>P6utWJCkAX(oRRr+|qvQ!S8<(IGhk~KosY0l&*H0d(o->IDOSf!8MTrSB z+Mj77s2Hj!aMxM44wbk4Bn*}mX0$jjJ((bDVu@;MJP6Zw-i1Cg%s9^Ql7XU8*7i|i z=6<;9mMbt}&j$`&iD0KB?5KPr3U;ijAmnz9_=ym7 zXc5X>RoGGHc0O!e+e=RCou$m5-TUYkl!3;1D0;7jP_NZr;`$lfB)|CfnvgrW@p|Jqz!*o<*uR58I`&{5qe1q zJL;bw|1&|v6s*dClp3N%86?RNIQP zFuT2p5U^iCojX${VMnPOf5G}y4+$w<;xh=IhfGm=s=|&#MK@u^+ArkTUL6e{bUNyV zQrF0UJ*965h3lvO!d=*aBg*=CJXFxGCA3zI)g)wXT#oV|3WFx6N}#0PAi1u3-b@}$ z-ts5PLrj=a_e>{YaK5@0cja{(P%+#8hK`cLjB4lQW)Zp<*`NvghdUi5VaD(Iv9N4aH6gld z^c;f2p`|EQRbj@iq9T~Tc7PnxJ6D$nEst(T$!T1I&86*xjPx$_-Zk1{%b3W{t0 zOQ@-k(IX^ov_iQJ9fI1Y?hyv&4*Qi*T5X7m+TsuG#e^AE&U_?y&i_T9FXh#_p@O%k zKnqD>M%nX|=5g1wXf-OmF&g^ptRxu9h#L^ZE-gk?FWn0hS_+`+o<6dcyv}?c?9bVX zDp_#>Mz*&S?Di`ga;K~$%-E5C0_LuILI~=TT0k&AWPy@V6=rNKx&zbJ{vdnz>KpN( z!BJn7s75xdDt%9gUq5{zcP;}iD9z(Zuy0lap}1m#F(G2(N|fzTBvd+8PH3Mix`>cd zy$}_+Wj8bv6K0e?(?f2UuVuoQQtLLOytbc%`jWzoBIgwsb62^@9u?Pk3_9<8M9`2K zZA$3PG(#0F4T0e;x1i0QuVh8}xl4GkEyo*`y&@9^w7(^o?w@AHos^O=V|{)COj%V& z@a!75l%RKL1!`!TFk?kgDU4k^M0V<(Z_b12M|YxnW?Y4ZrCo%u^%~2#vmV%pQaFAF zw$6G%$f}UHAoy>zMVSm8g_5Tp5bEZRTuw-?HbHr834*#}!irpKZyPV(L6m zj@#3rnxrseciK-YxGPxnJ1Vp>4qEN3A;`-}SrJ+?m!UG39)Nx=#n5=qcd~@M{%<^3 zo3jIzwBj;!Z|@-J>{qkq?)wyB#XazzfbYyUt{-ao^EyN)TsjJf&A zFlbdH!MtnY?*y4ct5A*8gc;L{DhS?d#T*F+y^BzyNB6+088={jX)oDjz4ki3q&ct! z)famn7SDQ3h^SC<;?8!X14?P=IBZXNOvsr#W<4RW+8kxJB@{}F2{Se%e

P|J9iX zz3O~W`rEUhh@>!MW!lewa2L1e50rCbBGlMfPv{*d>q02X{0$Yp^f0t(DI+NE5!pa! zn<&hfowExDthfeE+q=n9`!!wpQvDQR#+3ZC&~w#Gf?k)xM(&2f?NAldgc)Ot9ul0^ zj&dWY_nM-5qJm-Jj6xVz`h{%0e$FNyR2c9=wZ>+^tXXXY|B9*Z+?i}#hmstMhjj_H zgygx>n+YD(%Tc;p4#L3^!i?p~gXEa`1|B@ccF`1QC5wA zL;0Q02`%F$dJ-}-ZBTwoqo8riKLm+ABeoK1Ckitrl0pUJF!JR2Yn(Q372`e~>lT&l7%<)%VpGRUDfQBW86HY%8V*a;LO$1FHSM zB$$)XKnR>Gw})U>y%Ht8K*u-ESup=jKcVD7>*?NK&MW1(WpBSKr?=zWCjiNcHlIU&$=MG=&0|4Ob8nH$1` zDN}?QJ@Yf6-l}%O(1&UJxvL0YkBXfp%;;29M^IlY6H4g$yaH7ibqI#dxCgCEhsX-+ z4G-{O>(`yAtk|p2f7VBWNrgrjcaj_3P<8*EfyoIk2p)4M9VF;h+oA?NqG9<6VaAx` zk>Lc#`HN9%bwRM(y8sr97G?}hn=Sk#%WBb9lzd}4Z1HO)WR6pg;Lb1eca-tc6Hub1 zhEN+Qb(oMeQJB#^=K$1MaR+{X`<+}CqA&a;D|(7BqeK2>sJiMsq3eU%5$^KCT~Hy@ zgc&W1o)P5MPKY8jf3`+tL`6d18RZ0{(&0x5;_DZpYQ6@*#MnIOHmirMT`@D7FAcbE zMwR}T3ZoL53HEap|Kd)idJU@6;~2~vACzRIG1la%TDWN2A+;KwWL}5m|oN%bTq7-(#{g0d* zGCz(7gQf^Gn&;<0nN?kc#t$=2aF-Uo5#>Ein9-o9ks!KOKAuqZc@-)?>L_%X@qnON zDt404=VXE^{<;T7#NL3mv;Id`s?bj0OYN?nsGR@K!@z{s1hct6o#IZq+5y$z5eHL8 z2s3&mk4Yrx&tHZTsSAab-o-F(v@oM{+OMa1P-D>!RPXaFSmf712p^}G#GOs%A1KA8 zr(xTxdO~)f?B9fdiNcJgIftRtiu;86w<5`eln^77=M-T^z5Hu%$mSEd;)AB}ldRZq zca+mKVMg_$mxLbs$tm0wezrq}Ma4kt8C3*@(ots#txii&Szm*pe{3N%nbl90teBI^ zgLSUkP|5#gK#znrg6>?EbKDI+T8Aq4I0<7$2s1h+OQ#Xk=3An=pB{t--gjZ>XkkXH zG=uXzD8I-L)$;rb%=G&}@EbQHojc=9SCqukWLW#^IUy-<;st{HL}5mqoGAF+vVu_d zc0>jtI%E;bVTv%LYJNWKvgsw~f0%WVFNK6}L0L`{W|S*xAvD`7WpbDCc`eE}>Nqr- z@t7c9DtU=e<7AFX{JI~y#omV6v%ZlB{?W_g!BSTrRMdZ$pnbwSg38>VFLT%V$QhOA zkqCoF2s2tFkIyE^&i@V7^fUsddzTS>MvGh_7^W>ii7na%tDon>1ix;w>p0CEzND48 z3Ds|Q7M8qfB18r%T;49o9OM}_3UJ(KUr`+PsbfPe$R8B0c zx2z_lyd7Oc@C;ds(wid8ICQ-TR@e-XV?WHj&4W(i+fnM%gc*BowG#^MRg1X``@9}y z9d!yS%=nkkdROKSAN@OkeoHFgi!a7Au8F`AA0o!=8DtH;m2WtL9}(Ql>72@W9^DAg&#j9u67zUJ*y9|u7$M9ko2*#HU6X-5D*fp( z=;Qr}U^sg8BZ63(8LHYi1SUMc1zr8VlC{Rot>(f0i{7XbvrHKI>Mg-8aN1+;lqU)^ zc3eq-xt4W=pts{{2<9OxP%=}585^&c!Ze#9viFDiPk7KEd?!kDnlNM4tu8{my+$o} zE}u7|G^5VIKJ^!b;=A%s2@y`VDBG_`p;GJvLc7+;IzrArCa6HyAZYgA4Je)PKXQZa zY~d$asgFESULNUCe}pijNb*nh+*KO# zJ|Wg-_zQy52P2evcmV8C5oRpB^@$v2KeLH1S$}p%DMY2hR`q5=*4@c3x$}2ggEILV z10`dt2z6Q#&4lECmY_Ub_d(tN3gO_%esa04uJDtrm`B@Cjvg1G+6ZCB?!Q%9xGOMl zLWMp(39Y=J668mZdrfFLXNk%*4u^iv??Pk0|Hu;K=C|@-?L|LSl35OPfAxW&6FB1y zci-iN8OyFD!)VJ!g2UU1Z3NYjRj4jSVaELH6@(BQv9|=v4~tN8;d@}SiZElwtzNRP zy>>faGWxs)B_4Gi)~LTGB;Hke$DNy#14{eraX4`7F`-mzOa~$AA9IwwYbaFt?>6i_ z`Hh^X`|Eoi41VN;vhc`)vLl2UoBsZ}le=_-KTtkT6QQAZJwa@=><2>ix!+I;#)qNn z^D=^#pGX&>Us{;44G)G@D#DEMx4w{F?C13Gpyp>URNs*dSghVgh`2lTGk3O5 z>rhHx<6--;T0)MNbT1+BpXDes*Mm^{zY^GRa*&*=Yask2%j?mfD1DDBP-KKKW98rK zecZ(vxT2h&{tY#}pA&i|CVn9lowGrO8%IH#=l>8C{YLZ?+N6aUvo8k10JD5(`sy=T zDsa|UzEm$K%$RZ|4SHI(5cJ+o8Q^Yc-&$0KqA+9Z^=g8XjpR3i`iG^co`d^gp^7kL z*sTGwwY{G3lPrbL+fl7YF2OAIcLe{tsz10hadJjUemw>2j{QqW)*3%Z@c73HrRy32 z2M6yF%1;jaj}W7~0Ok0|AF6re!fsJv#)7|p8RAQ!2Afb;Pg9`0cN71=zePe}*uu#Z zW{t``w;JhZ{1-HSUdfGwpSTE5)=CRQCSBYM-OX-7omYM2@4Iz|^JJNvFl6+V3(&#x z4MFv-vM6_5`y5gEio%c~*Pjq9ZN`ou$bGOtH6ILv87jh%zPEmmjqK+MKgkmR?2D>7 zk_{8pI|*)gr;p@L+i3%8pg#$g9%~>(X~~Hb?EhJbQgJ;3I|s`Nc_&3j5rTD%Q5KJO zLs^gOut`)HGW~BY3BKfG;DIuHdJc+tzamsiOc~8x!Z~}Ct8pyUdj5#e|L15)LW#66 zWaPyVXlGUgm0x`&ckG@!h6i)ygdu~jWI}Vxc7n{?X;R!Z?pu#aQxt~uzFtQ#u#p){ z5dE+MRdw(Xj8|dEQ7*TJ$eQ+s(mdGLvlCT(!GwQ1N*Uq3us8IYPFyFk`^Q1JKm$4wQQJom{_LU-(H@ikvW` z=atJ)&+Yy`D+6zo{L_oDW!qaqro^^`BRS>zW&?rtMg zkeo21`IT!>#_|)nu}xF>NmklEca*oHFr&fsmjqFp$uqgD>a;_}AB=%6D#DDKw?=6Z z`qnH#754ih|2EL!K@9^xRB!DSShVc}AzWg{9PVt+xuO(}lVMxK zb3*o?6Lkpz(!z|U7o(t*Sp}iKdBj{o%I-xdPdQ;my({@}Xn8NWqHUHQUy9wg1?8kD z%&307h0wE7=~wOwJJ+JZ4jzZrD#DBkw6!MEx6C@~+-I9;{pE zgG%nd1U-(uBj{@VY{1=MxihMKLn4eB{FmT(a{PRPn(l9??y3k_u(^y7Dk@?~u=;xe zO5R`>Y^lwKncKR_eiE7s_>%FtO(+TDv#_?IiIDWCf)RJ_(!z{77yp9ampmktHH$AK zMDI35Imih!s$RJXyOw_;=eOw?^I*t6FO;RCFr(b{H-zSuQ-y!Y%II`N`5ugiMk>OL z;02+weCG z9&8|3oRnKakkz$8HB}vf>6`x{_=t`$BN+Z|j1n^lgw?gzVZyf0WLJqN@ zLsZT>e;C+*6`CFUNS4<6#fmRAl)Ir)H>5zX!50MmlaqeqPDFP#sH-WE%+`;U`&&=eDBS7G8ka4XuQLKb38`GnE!*l)88V)-QQN zNNJW@Mey8hfzp!`W*o}C11pyQAjh`p3qQ$n+UJW>R}^OK$$L*ITseI;cVV3_DC>hs zP(ekQv9)l59U*JYN|b+3Bs4ivPLNa=T|=lVS%^wrw;Ou&=Rw_LJ>j)0JSD;kogc-ZCOJV-~6qRx%i2c0`RXy(jOsFk}uG_wowIuX6@?d}J4phm)%P_K`gJAcknj3e@(!z`# z8K+_Hl4pdVX4y>y^WD}c898Cb#_aoqwB^Iy3Epi+D1&_gP*hQvu`2HqIez6#;U`%x zo$e^jgQ>7@sxV`5;baf)BG#-y+4jUhr6W~@_URH^2stH7P=V|AL9_lsD1EG-+%QvD z_(@i3`8Jf-hKo>tu#F&cQe`W5m2;d>aaAXw^X8`n4bgF4gx(}eRMEU}7+!l9+HCue ztSB*m8xOXn`k}HH=D>i44+PUcXLxfbB`wTYpOFkxmNXJPnp zh+Qt`LvU(agi_zP2lh-BW-QF>C5Nrl-oclwJGY<|4xWdtQ-v9`3YC1h^IzkDGU+)E zC67EN)J-4rCn34S9Obbt6zcZhhJ!KR$mKJC_2a>qavzlAhAgNy_>RyW|MO1n3g-NQ z3av_nR-5Yy@}jc-gqEb=P?_@%L%-TGg7G$yT?7dUVaD3jT`+0kHR#^ZP1gBSGk`CB zA1lmQmT?wFFL_CDXja(Go$78oRM!My#{BGugplQ<0tuFFrYO06!LWIpOciEKESws|o!gpqDD9qjI1pJ&D4j07mk?F59A&@mAXMouft@jf z%!8{mT{wK;}!xbny*hy%LR~P;zD}9bD%BSjYXt?<~K}>XF2%$R329+=`3cA+* zL(tkbVn3mOlrUpSY9NeUm=En5K9iOIoE6HKI>rh!=4Pb9pd~E?^X4fBxRcqv7S%XG zm@zH8n&7=$GK^r*wiG40Z$GS>EX)|6H$Zk-sdtbEH9NPX`ob>3;;F)n5rwMZ+}W;i zMk)22g6)z25^|=GKST&Du|k=xi-6Mo_XrI!!y*W&GZ&z|%Kf4KhFmBz_>o*0|I1;% z6gOuR%DE~9YHV&I^bS{uRuSI@Fp~E=p#%0 zsT0M6^<#w@Q!*|<&n0gNdd_L{&@>W{k~#LU39>HkzQ`W`XJn4ugf0g&D*0 zevqwK&J%u;rO@e%Y7NVVSyP1>{R^kZaA&e+14^<-4KUhU>8VM-RCmUh6nt3Z3JDvZ^`<5iFO>BoO4bMC=Y_N8wz04kN=U= z<7W%M$MTu86=hhJ4#hUN5~_zQCv%sO^gGIR-U+BxTSMsgk~%{u870gZnR)=)ExZGj z8@`h}eD#IjW95z&W(>->49%CkC&)Cbo#n1Ezy*~yL734y`x(Jt`Gi!0Xqz>vDmW6x zPZnl$$s2x-pt*7(s_%UOEDp_&q6a``Mm&zn`K!r-GYY_c$;b)MK|g2GA@RO|aaFe~f^^q(rs zXi})1&6gzCc%te)orlShuL&O0f4ah*ZixeG&@m2{e|b!Zi5ZhaaGbderB)sayIqQ5 z!H;j`(D+}k@}Sk69Vq##EZE}SLC74gmdl-A(jO?}d8eU7Z9SpZOZFNeX_PRdd+K4R zv+zFQ`!kU|LYc1-DtfFiqeI3usJi46x$C8-@O!NM0C!Z#1Yt(Y?3V<&<&*QdYkp&g z$_S2uzLSL+jq*m_Ac(JAf~t8R3=_i&q1#kpM(x5m1w1%lw+&VLDFa4Dwh`>7tK8&H zrDPqd)A1zC`%+5?j*%`TSj@CU$(A34O)hs~`j0`fPrSh`9yFZehZ3v00;}CW5E6#Z zDB{jF$rYtFFB$efc}^(tns}QKIZBw(E;R}&FRUPRJR4C=$n{->3K}cSXr7S|WtQ}k z8(+>6evg$Fum$BkL734XyM-XST&aY+syAy<@xjNT%Vc3j%{a`YxX2Rs-X;DvIUAk3(q{f5wEF}0Gr z!Z(hnu;6%TJz1DhA#dzMLaX(1RMz`2=pR-BO{NMnN*2zm;=ww*KT*k_vY|(0CqZ|* z`XlZJ?`%MoJN^x0zBCXVW8|s{YBOz6-S>~c0+)XXp+7_)6RhHmQSx&FVavnoFw^}r z*>Cu)8op$lXozz(P-Kd&S_H1-5A=-B-%3-W9qiRMG z>@po7=f9l$lm|lswxcX32s6rMw-cHzRO`6Qc(Wem8+-~HO%`Sp&y)F=P-AU{N_>9^ zx`o{%XipthPZ+pqh$^-7hf$xdLi@;%WR>Z^JmX89cid2Ujwvws%L{@<%%leHWM{5M zHQkSf=`NK7pC2Qi6Aa@Qqr~Ryh1CxWV1j!e*>$*%@Ovz+q^+oa{R^<Erj-)3sE_CyJ6s`JZKi# zLzbSd^_nj=+}Vsubvy^XzBCi`V-#Dt6Pam`s=R*;#<@HqIRB7%L(qsfL-p!}z@mq@ zV7U8Nvd!?h!tb#Zle|%F`k65M$y-8z*R;3XnT`@>luAv2^+t7slxO4G37)@g+>Q=VQ)5~tb@-$g~`H_PtNs=T4#GY(uh56jIS6XG5# zb`x9z?NORz#EhMn?-6cT4u4CCcxr~S4GDotzlYqs+oDLg`6j z#+vJs-f@>=zaHiFG9KzjRTKKvWqJu^x0az!Ivs+}Z;GHsY#+Hp%TRoeRd9P3D%|x) zX!E{}pct>w$6b@I8!GeOSs38)C&BcS{0D+mk}c|iK{QOhe+PPceZ4$ly3MBuvbOQSaMzMGj}oeE-1&B-$J#hhlJWJvqu|t~3wY zKl4MWglEB46|rT`^{Jz{3%1{gvUr&UWus~db?WjmgtS{LQ9e#bq2ZfSICy-JT%l$1 z1rNsG-h*;=y$ZG7cM!TxYs+$1sOyP}y!Rcn^LRo~{-iXT&~nBBm2Gem2HpRSVD3Fa zjvzB!Y*~9g6s8*AfZmVZk`4T3kKvaFq{Wuy7cyX+Sqs7C@nm`KGy}gvb;^k?Z(M#r zh_I9$OR#;q9HkU;7&a@4EwirllLKvx6nN0=+1DuP@Sk9fikLCwy5>0UyzJdk`Y+GH z{v&@8%GAdx5>DQ-LpeLeK#ezd2_46WjVBanEkcFg4u&?a`B3rwYjV@+nM(XprtUUW zz`b;6>d{1y`ZRF@cMs02LnRxWhMxDU2zuVq%7i`%F=OfZgD}?k7Ib>lOIG(Yn8<_e z(qhK^3zuP-*)xLmV|5ko6armQjdEhfjLVM*{+44V5lo)ipd>@0V6~!{G3g3>jP|fu zpvr@~&jL}s2XkSGikLCx`t-@%IofYQsl7~v&yO?^^3;{o2%)#upsbuuK>0Tngofji zQwSHe%u#-~_d{dXpP|J2ZgSOWJ#~I5QP&6Me(w_0@%V?(GoU(^yW%s>sAz)(=y3lJ zf~xly8iY0pF=Oue2pD2q3@smhAj|p9`;rIirNxZt7qX$R*>i%?<1aP28``}IRVgQC zOt@S}aI;jHM$mq`3e~;u7%Wm`#!*pMhRF6drdm9x@@y}v_24y_qatPuzOFN!I}3X+ zl8Ya#6FeMc@lJx6?y z6{))uWp^(VDtojNS_Y=f`Fvk+KwumGm0h#9@E&k`SG>DzBd^|xPyWk;F`C)HIAxO2X>9;M;*4eaQu zCKMc(nM(-QvO?M1J_HqAe}PTCedNs3hT?;)09}8S>AfGJl*fOB2Ll@OxJy3chVnE> zhI;q^B=l_?YeXoO5HrS}kA_ahWl;UmXL9?_h4XnZUs}uY;GqM4@_IkUG$mFC3O@IU!d<%ouY1IJ7jrN056Y zv4l{+(+rg^EoSt+a2*<%y&(_P>xvJuDtCLM66C~;ZkL}Dv@Its<*vKQ301T&9!4pO z8SSr(vLvY3EJL-nhQpkLMKD-J%xH0a?lK;fwcmxRYtMpdN7@KJ>Kazu8Q$868eDf4 zR&>=8;*ZNOC%9_aqO@)wgj+25LsEB$V%5f{K$CGrC;30X5Bf$es1GZTY1e zyLX@>%xHb#7E~~MPj0L?aN@y?-F_&4IWeQj zVcmD|Vpjvf@3_(ig0YqZO5*kjShe9dLSnC^E5ZG=IZ8)26!w(=42w6vB}WhFZRD36 z&iJ5I4KiR`)jx#XZIj)&3y}~rTAojUa>fq`^)+8?BBbwJj`EcjGa6kehC_?`$(8l< z+<7oz_tz*lIWeR5<>!R%rJ9?$D{9(=irRM$+AE3~Rjw#_5L#E*p>kS}!Qg{;2^K2D zwh&~mo1*Gg2g9`XYtZM&YqFvGOizAku+R%tu`V6Pcl}LpJw9L3LI|=>yi&15(_rsIz1mOP1{)U* zku&NS2J)c)?!71zIWeQ;MDvEdvlb72qh~VQ8BHjpyR>c z32G`M_Y$7xTcGk*hr!VHo6zdWJF>j`9PvR`L!mF~;<`-e*Yz*K`1q84+(~F{Kvk6_ z!o&?V1oz(2Aq1V%D^NW%kHF&c5*WRCfb1|Ze?JeZp4p9Rn{x%`R<#pCwrPcOXDK0O zlskVO*3W-TNUu>mK=9pZk1~=LGY+NSBUCOL9!5y0H$%DY4uRToV#e;wH{_zFv&08k zQBB?``+XOoilUgYHD^*dcR4H8qk>!Gp~b;!f~<#`A-R9H7cjMv)<{1QjiuyHm2VvWbmhj1pj(Vl*#UJC@Cj~ ztj>H-PFgzm8y@s%@O9Qxmw0S+No2*qQ{@4%QN$Pn7?bke9y_6}tK; zv}!Mf@<#^A4O2`KdGKQ49+cm@tI)WsgCKES`wVwg(>zg$CEr2!4NnL+E; z+W8e~Xr!32GW`J|VUcVK!L5EdN_+QV*gaazSd`gMj#_GTo(JulzDB9+`w6y=6Eo)I zXr^)(yuux2(RvQb9{h_?H*wr|gtUA+l+WrIXxM(2Fc>*3jZiT~%otx73|-gdL#?jY zR;*+exY1>edCF#&^LlZ%{cVaqsEph8m*)vbWpzCx>X*3F-06zy94ThJk^YDfv1m*N!M5H8rL;Q= zHjfrFW@UaN2QFP8KFKm`3Pefo%Y`-L#EdC9(=)mATCoMC-6O}F#PUf#c zIj=qeHQFl(9g&h*gn}tz#_+=Z&}QAwP_e6<+!UktBfpe6%?A}watWGl_=h0XtD4Q- zgE(hY^2`M2S^ft>Z}S&F5&Aw_p-K}YVC7uHNjo2dE= zcRu;+QHHC(frBrq2^Ent#f11NVn)}(Lr`noFR<%%AGt8bP<)aVIn5tsSMnoN-tZrx z<()oP#UWfVaJPygo4P?6@>69Vn&<7 zNT|531U9|?NY0Fz{~HemOxukzEy;mW8`=pE-c76IE;(*9%5&ylSPxN zxs$ASLRAOE!=%w-Mvu%Mgo}~#H3YvYVn*Y_qflbq9a!~xkenD}@`wlBrv;&OO0L2l zmsf=1ciOewMaOxf9A>6L)$%8VHunjC5^_H}ph6N)Ld!Xo1i7jab%c7aC8+daVn*MT z8_;Ne4|(X}?7#S>O5Yu*gpp!KxAYc*_9C@GXZe|m|6co1&Wj^x)lqqi_)VixY z1IN?mnAB0n3&Ne_cMa-yJcmg&#{zk}rE%_fIAjTYJIxQ4R zmHZ4JxV$ANztelpFL}oKp!8;Dz`ncx5K7%Azu+$Rqch4WF#)R2c|d6Y{fl-&zSnY8 z*f24pbxJW*nBPxsd^qnV4`%p&jq)EUW;98EPLN!r*}+})Uz<=#0q3B{XfdO1rot;i zuZ10|bZ3=%=`Bm3=MltwwgHeHNPaE?}cixONSR<{7vwS zoY=*k@f0zmL}47Pa;zdGz8?98;2yISr8Dgy?D@427Q6J4qu((bo_%5`C8^9!7j!M zr93SHw*2}F%y#)e4ti%OKFKnV^GC_d%!ajh|0ATjfBA_!?~iUMgT!PwpkGHQ|9$Kr zAEtzzC;;0%6M`X(1r=VBx@7(B*8!5p}|4$aEGV3sS^52`#IqV%-W8xg~Nmj=- zUsQozCJcY^FTo~qiX?Z6Q^b%>Hxpr|V+|qT_2>}<)0h<~scA>xgI`NvvdaM3^WA(Y z9@LB5jq2060!#0<6Jp)9Msnx$aWhIi@jPtTe@w{#T~V44=4Fqv9wvrVNV!L7G#WmN zknzwABtEI-k3G!Hgh+l9Jl_apRs(MB+i)R5y&Vu~2D>gHLP z==dkW{k8lUf=-Mrsz);#7XNw&M!S3_JG?WI=RwuDAXJ-99?ZS_iV)&HV=Q-;A3afW ziD|H2zmbss`ve7ouh&|X(J(RN(78%NrIFM)Lc+r(C^z2&Pn7?)A8d1m&33s20uRF#FeXLXeAu z3c>u{B9u(rK3J=B9j4yxB73{*PU4pgK5jz|e0u?w>pvyL{jQ?Qor{+fN^_W)vGd%0 z!VRNQlL-+IEm5|<;ZSL$n6df$_v9?oxoSKZ_?I8bEFcR?j}|l5{4jM2cPSPdQC@#1 zLH*!bLcfB%I-%^Rm8g@}N1^k-rBEYmkldkcB0k9~xV8rsZg&;hyyze(Mrvzt*Q6$9 z%)I#>3~+oxFnz7`B|$320rf!hBuxJGH-e|j2u*_CJ9AXusZdy|a|6cSeM@$7pFNFV zQvc|KYX3F^=Igf*!hWBu#hta+S15&HV#daE4+t4XveOCv50|4%d=Ep(kz&T`@B7I~ zrbgO4=<(OrDBXacV6Uv0vE&EM8QjHKxT75ZJ_pr;{~|nB7^g$X`^gR!Y8?Zu{=G|( z4;wa<(4Z`4ym&1b`q||};}@^V5|J}?`K2l~F=OJ*bm;EbM9_IXaTa$y$JU{WHBZCn zU#kcXF4B4g)ptu#ZKn>xT%B7m2{H?hzruXv`df+d~_awr>>dmJ%}-egBCZWxBwC2krj~M5zSi!d6)^W6lrL=W-Wp zu?1!EcPf+(ZXnbtC>avcep-X_u|5F}|E(Yl9*~?zs8AL&#$VeHUG08`S}(fEU5E9I z_@zQMF=OP-OVG~oAA<60)%o1D9CJoxYbL;;U;iMOyL@3xka=f?sy!6}Q+0}=_uUU< z1NV6gcyOTKA65QsHjLAMPH_4COB3!iy*8mbhmzrqb9ICWBZY+o+lQ-AO1{TnvlKIq z%KCnY9B684%7bQq?L|okT!S^TV#bsobQW>vW#NU=|NDE`zxQuKnZiUf!pWZ;QO?$J zP~+d<2^|MUE+!Nxiy6bOg+Uv;La6xS9l7c79Pvq3rka>B;ASQ?b!;U_y`CceTGoSO z8&JubVn)wjYY2KSau$TXo)xIlQ%7K|&TZ&)cYv(!ZY(~@YVY5T%K!EX4AXx}u>M`k zk~;-24^-pOd6;qTF~Q$x{4#>cLwl5@Z!D~q5;G=!KirDoVY(Ql`&S6;{ki~_$ch_~%v~HndTgHY!m~Z0dgVN2ufYAEBpHVN3`r(AHy%$~_egLv+fZ<=xL@IroLD_`|L54??AXn+JXMI|)X=&#>ceXzNx~ zkwS;(Ov8n5|NT_930=u4nBo`i@ z@5qCZYGPBnn>kR~v7OM;Ic*(x*~i4c+XQKfP0fF;C&;)cIuUAnR-;l+9f#gJ_Xq}e zCDs!LHkqNy`}e`PZ?8ia{WoOI-*v_JS)E(GQ8$Jzz=(5C3ARR)zT!^lp%bckM?B1u z5}O8oKgxw*X1WX|{Z}}w`MLq52!MVAbU>QjV_(4?49sahH6| z6XmHXX4L!j38C++3GRf_9tTwHsgux2r;?z4cZB#$TJ4*bpz`}eVc54fptXJvS>gBD z9{f_{)*Yyfp$zDMu7zM?q_%}S$%kK|s&{+~lcdCq9^cD)5_CpoI6wLX$6b8$R87+QTu!{#}E%u=58gpUV-VTC~g0??* zhCgjV4KDuk8fGiqHs0lQZHMkstP89<0UY>u*16EiB`{28{aeM`>n z)Z5K31s(H2nQMv}Wq$pKQ2W*7K<-j|oKfDV5}<+31HwSr7kdcho0g;E`Xivrx5ZFX zzn|P$IWLF@Z*2V<6)}_zZO=U?C>d$)<*xa`CREmrb1+a!%xLz#LNGzv)DBfscMPU{ zeV5=RJ8U08{|8f4|B_%>)_4t`-20mBtS~c#U()!=3)Qha9Tv3wO$a|QaX)u9%3?;v zYjLn?RTUxg`N&X0z~QARQ#CQ8)XhTpU~MlsxpR*AKFjmiPL!Uen6a<8l~DSXdKh=H zJsVI?r_MliokxWBGP#3<{7p8fu>K>^`djhuHVXQm$c>f8;`^+Ot$`^2p)1hj+)IL_ z(RA?_wyGa^pptf^LJui1qwe?PBM7|<*Pu%3VqwhJ6$D3F$-@M-9~PsYFWC?C8Vg|P z-fps$f?g!QB>&TPRKxO1@M23d!S8_T5$=qY#f%cyzJXP%{vaeimx&^{AGSj2sEHYS zZu|m^*M1;JcN&WCvmB23qf|A;jBUmL5puu!GMc-P9ygTbsbnapQ%9&T8+(k9zG)T8 zw?7&heOm^HW)G1oD;LJ_V8YhDD7T?JsC}-J&^=G*ICn)4wxXhTd=KrV#EdH6E5{OA z7doPH>Q2GnuYV_4$c{Wgko~~|RktJzrZwJ#K6~Gh`A=Qq`>eriUsT2NOc>wtFTwS| zlvCVkDT^7q@)BX;sv1J%^U-kxyTdC`%4%Z9mK!B7d+h)@sB`{l9yC9;8zrMDW~?o4 zC!~I*70;b_&t{atsq=7P#$!Txnc_EuxJ~vbm;U2W^V@rb&e_8g2sbLtP!U^0pzY9g zsC4cPxq03!@qJd-18-E|j*HMtO3Wzz{iHlt$P-ROF z!IOZj6R`;10He%XZc zgs@F(QP%w@p~AP7gvQxYsf3KmB`E)`2cXH&4Jdi8hg>~R|2uvu>A?<^$BrMMu9TRu zH%%>#yOM=2sF=EMq2t#N32L&Vzb8D;utw!AiG-nzze20MAIb6x^V4~-Av*wdad{5( zYk5I1J}~V9cM{5C#;Uv&n7FE*;QoBvMS{-Z)u^7yV#eYd<%H<95|;=Lor_Sa$M(Ut zFU5?x#a-l(uXHo`CCi>|D7jM?VEv4zg!D3%AGq_~AZQ)X{)xM;31Y^=yzgM-swV`y z=Sn#Q<--oBmdRqq>>Ixkg4T|>LNM<%N68!ug|%Ob8B>ejlD)s0oy#v7^!T6#PG-RJ z87+jkvdLGubJ_G2O0z!!b|yR^+?Xw!M~J9gjDODJA?I$n8{29*(VP)#KxU=5mic;u50~-?_5i(|v zDI)k++MrCfMnTD;+ps$M6FF(#0`Yy8$AdtW?v7m8J3`D@k~Y1VyO@PrP>yw}Q0?mm z!t*bbekJ5(tU-k?IRUL2D+uy?B})hmW5tXYv-d;4n_56|7Jqj7pjlAJDIT_<-Ft@sL@zW=m?UjB@~PmGlpj$f;P*4fr>4Cmq3Y@kL0S*`A>N;ae|o9Jue69tZFCp{5P$cyW)t=sOZUJ zMu!{q1l6^Q{}9?YFd^Y7!EN@WR_?SbolxDL@v!LgeL__7sAmNGdCO2L55i%qPZ7)+A!ZCto7={N z77KTwWb3kEoqro4?F)_nxbw-_h%#Jq77jkCB~%2-KPSYG6*IbKAB9@W@4&8q2FZn? zCNFp}a)OxAF7GN-UiFI5@}G7)ci9o1sG!MWM)Mm_2r_FYyd>1VazLd6DiWv=x zM|2PdoR^@=--g1tlQ*Etj2^OP+3Z*RQm5Mv)Qt}rFe0IaU^`o_lRKr#uTagN-@>fV z4+w$Dvabnd^Q=+Q4-Uf`pI>3h2r;8qno$=I>MsmH_5b-3Ec1UsIQhl2H{3aAxT7?d zq`;0Re-R3T#&r|I$BG$kvSXm)@^V7cKf~Sga2ms@Jq=N z+fbg9#f*A4nh1RkD(|=}eYFl18+{r&eJN&CFP82lv^!g(^4}hWVJB}v>lyFK3S|cG zd9cyV50&xZGW1V)MlhK@wU0Z=N>^01XA(^M{D|O@Jmv#Iciu`=?;laH#HSR-j1V(A zrY-2_LA8Z@P|yF&g?auRgwQXvKXPZ4u>~c+*TnIQs;YSpFNK z>L1BZgv3yDl=}oRqfXw>u*dc-x%fZ5L4GMZ!UyFrSZW^5$c_nqtf3-K;M(a&}c?Ke_3nj&b(n3lRixwRq6IMGT}oubW3>7rS@!1 z2`;;T+k`6eJO`sb*AeWK6^0X3=GmcI|2PJ7eC`s0M+}oBSfrVvWETd*x<9YMH2>FR zpD$*P;1Av~!wWTNo(?OX{7r}tnkdDc>sT?QR(2fhvZ^8!{xfnUAu@C+%5H)fQaP^> zw%GQPv;UhTKF$h?*oiWqEQXZ1(MqUwP#?uz>Z=VX@8~no;7c*&z%Oz#gmPyaRNUJm z(Bkx$};(_&Do^TOcap!xdUf#i)Lh{jluM0(jEDo9z6B-WYyKBV#+N!~7B~c+yM=4^oxq z&StC_QZf4**kttwA@d)Zv4nt7E0pO3F{D)9FYtlw2XgX%h6+6B8R3u8n=EGR`}sdY zsl%7!xQl(|hH{EdhU#C68QXsut4PRqUWE#K8x5^bmO+IXL*&Lg3&->=n%0Rr8-&6*!FWfA=g1`GIt@bHlr+~&qKK{#f_Tl($`izlEm!ZrY}?P|p#My?c+fl|2qiOF z%vk&LD?+NnjOpBYzw$&GM5n<44KZW+FB7y0an5T|E^kjl&6AacPHm|fgd2C3pd#E3 zK-&*Dpi)8)xmizNe4Lf_+YVHq=MT{Aa|=N_S#2hFHHI#zls~?OUOo>A`Xff`68gWh zMwOXF!jpgg3Z4Bwk~O}VKZ^%DE(M?p%yVG)lNSV=plN#CDUKC0HvO0aGp*_g0soAf zO)w2zjgp!mW_)n9oRDlQp-=GqZxKo_Vjt{N6*HFp+(nLc(4E6CIlbD3Qjfj>+cm_D z`M;iTpyZ-6V7-QzG5wdxrrh~De}yu7n*fJSJRnqR%Pt}$ z+*yutb2|*RKNQ36Z~DnadPZhE81>uND0|PJpvvdxgx0f~i@D1&bVmjMaSmGe{6&x* zG0vP&_njRo%_Ii;{CSsP=s#=;VNgcQSaB&B#+&Cu*C(&ZT0t`{_@yp+F=OG6=`hl& ziD387#HHLRhpt1lD2W-fuT~L)Y^5y;=Kn24$wVB4wW?yq)Sr9F-VO%Kc+lY0PSn8B z%dlKS%oz8Jx)paW&aNoUw`XAIiARJR+GCazBJS9rY~7-u(udoy`I}GVEWHKdxC6#OlW#6qqoisTCo&T&LX!uL6By`A# z84E7$hvDWwLz^evWW^voTYjlYUd)*J<0Tki^$)@HAJtXdNrgJ29w>f$ZckZ#569zw$@5AI*mO8e+zxsK2vBWApKDGd6V7eeDF@5mBCbHvA4Rq|rS z#2+)EyHzVe=btI-x$D`#0adIdW{kdCLvXN_b0(<%w*u96=m^YJ6*GqXJV3T|Fcu$Y z$-UZ*sy})KrfY~9eSgt%;m*j}12xor9#)=sOi0ikzk%R(#~!8a77M%k?h%T<8SYAm z(p!wO|1AWncox9c!8hcbv$Mp%J$dbWzbgub@bsM15nVXUf{(dlQ2 zodk6UGgN!WKA3;>ItXyQx3>}UBsBuK3;k{*%F5&{ zl>f7q(6CE>H{qg;n9=XjQD|&_2TD8{Bv8=k5{3N)hj|zv-TeDiuZe> zqLsvq4p*NLRBa~&5!%`uP`QUr!Vpz4qvg*d_7dbAmZ0i8LSg#R8_-un%xLt>>|lOr zX#EaUWp@TlIMG6I(^lKZo%WrtP~989g++Z22vOh2h7j!atWhez9fqx2euX)M{p8@Y zM*Dft!Y}|O`^QhPZu<*Dn&h-l?tH#;M;V%=z`@$T2o<}=9U#QZh#6fk#Xv3daza8ikNHs9s*BvxJo6yGl)Zl&Do9DpXnwVcAY-c%&RuQWI#lYR)6iR0 z%xLhl^dZ8)8cS4p$3Yl(^cHl{5Ho82Vi3WDo$LKjH@Yvwh!f8Uw%StQVLTBKasu8E;z!2`i6T@{nfd!Y8uk3T~xtGDC_&3ec9 zrR4oSC{HCZqu$kj2z@Ij$8cBL=8TFxlmMMn#f<7de{r19zGgWpzas*M9W92|8e&F; zU*^T~VB`9)Q5oIY(Er49f{C`~3GO8CY(iCUJO`8d>Ifd+D4Znd>e-=sD~`dEEq7te z;1JpItm!EpR5J`lJ+Hn7^R~Yxgi6kgI zUPY*C7#UAU+_MzrE-z-(`LPi8EbAo~H_s8DW<~GciE>a9Gpb%~CA6(nPv9=MZ38Og z&>3i{DrS`XS?*gx{TdroddCsyd-OIm(hxHa6&Z_9vntmIq7u5VK(`Yw3EJAz&v4gW z>VYcSm5>X>ko9vte`wAz;r6l&QR!QR>GM_+Z%pIk|cM4?O6(e>X}`NzB-n+fFE5sg=oH zY};m()1mWFT~*B3UZ8lHkiW(r71nVaS|7bfP|z5jMQAKCLuITFf&Sgsp~;CiWJ&E= z;?u0^Qg2k!#*5IS?5BsR$B65RLLqIBfNj6GR* zVDYlgOeMQJ!IU|p|kTy@0<)Jhvrz&QwFPLzRkiKRu%D3Yr zG&)*I7@8`TPpB+ff=XC_0J?SGfZ8W|$lcTRuk%Yqr8`hj8-IZIeJuo)Z`2C7Yn|nS z%BlDk25)&tuo#s6nIL=C8dWzp5~fxE3VpVJBpXW3zrllpsR5{p1vxOj_65Op*R-45 zX~~EgyDp}{!o~H3$cAx+1iL+}QOfdS#+IydLiRF=TZEwIMJV(A`=E@Hn6Wmui=4Vr zw}@ZzZrg@3ICKFHOcFDe7pVNgUECTcluO5HsCo20p>yh}V#19gOH{=AaA@0o3o4y> zPi~$*_g5awD)mDJZp?yaea{He-%Ksxu4dLoR7yn>^x9HO&>tLgo6w)M5>+<$C_GtR z3Z1tPk~JhvN_ns&bq}gw!BrSu+d;6|rG1AxMHw+;)5Y&#=He%WfCi;9g6SRyl$5-f z@j=#agydx-?h-tk%~5*$Lt)=|F=J`&TXO8m+2#C_Q=1P;{ZIyMpCo3?FPMCfyRbE1 zp{zR+pu*7ygvP0|6@-kUjn}N? zu4I-wDyHHbblmb6L2Xdscf#`|J5=7>7#Lc8mteJhSQSBDQq0(p8VoNk$cKKlugS){ zX5QzQBxJ;lRTtA?;^HQPd&9(P?sWF7L-mXiGZtr65u%q#|3PqQUW!uPe-O5f7c=JO z_L4(Z8i-G`EZcUX_T!x$|A)iZbdr1Bap>5h|ySsUaj3*`VCkM?vlG z+ps(K6S-*m0`X~9RB0f}eq%0F>3d0Njh|l2UCyj6sNjlJXtAY%AUinzPeNVN8dTcc z6VRu+f?&8^vW_q~T+CRJx*x_b_!+v^c9XSs>HWnob&V1;7GAsrBNzWeuxn7Q=T3Q# zGpc2bm@zx+4?@tgFCG)jo2^hX`y*iOcrjyY?gz5>%6SbuXwc@58VJvZ<&(sWaRpyK z;m&2vCX{AJGVF}1Bixv(&`5|VT7|M*e+(*hm%--PA#&Dq)4zEzuyilVY~wX3-PcK| ziPvf3E@hS%%B$jgsK4cJLjNb_r-ZU3N7Tu=anQN?cY?@pOeW>t+6GbUdA5xOt_kD$|_ z(ZO9$pc|@qjF>Sx>raBivazoSs?D~jwvcF;J6_BflKYu#xpHAA56ZO#q3Xl)VEQC6 zqi?~C*W4Mc*@_x^nFcGP8VLzgmAeRTMQc&o>rcV%HoeVF=J3xJ;8jL;(LNj^J-LW z$Z?oDUd-s7E73O1~lt_ItDu%05l~ z#NElHjVR~2XQ4)QEuq6(evnWwT+A4rdKB6$xC0ey2gyx-CZBmQlRv-WUmX{(Les^s z2vQB&L)<+G^h70(5i@#bJ>jom=`EWuZ0V#UhNJqPIv`6!PQuvnVo0al5fa>}uUvv^ ze-;Y!!*9T_Nn%Lrg4x6QErm5ZP>nA$U`A96!GEfnBzGo7U!f$|e+#SMJRl^+%8np- zOt(hqmL7(^uD`;P_x zoHXI$a51D`Y78`9P)?Aj9X5(k<+lixI7$rZelZ{FEbb!rJf0~&%_!6AD;ROQRd zFd^z0!ENePIqtNJTv6RlNwDb6BSKW{m@x$V=_^qxrBSfewG`&O|3nUsUm!lsvY538 zC0mgT>pVILX`iN#<<2K*3(9crcX0521EIoONr4bQT+Ha2dID-K_>Iu@NOByZ(9awd zIZDiEckySay!b7-<*}Y3zmy&5g9;iWW;D z6($nQrrV*UOOL@C*Sj#~{SetJ-c*GL^=Ad6`tMzXWgf2yCqL;-;?6nA3#BnP9d_LR zn^53AQI!xrT+C>b8V3~@R1umUjhsx#^jnGw7$s&jy;umP7Wa}LJf0&y%}Ng3iSisH zX4K1SCG=UUPvNfg=>}A6$QkG~Ud*VTE2mCqx3NLxKRW`$!f!+CNn%EY0%P%MR-=6& zD&yr9=pXfxU@~>O26vK09;j-kRG9SUF~K8t{FemX>1$BEw_{<6YXu?Z{cuf!WBg*2 z+N}NX`Mm;|=h00L{WNPDzhsrP9VI{a5^T8NOt|Q+s>Pk(a51BC>Nikg!5@UGM>5k1 ziGEfn_fcX-or}M~9U*K)2!$~f0V-*F{5hMe}p#6FK2L<`_v5;5|Rup$BP-| za>wcr>TOn`(w{{`-|#YMG)c@jbbaAW9;~$Ai%NKz2i>AN3EES2bh+!kwG~z5^gWDv z(@3z7Rh~sqneK>cy?qMixc*KEelMj*u!y%n$<7Lcb@y(>;=pHKR;`6a_7U(}#M zCak#sFCpH0iavL)!^MnRsfn=5xQ0;pX!IOHq~8jZ-6%1m^2HL^Vm3g|emviR2ZI84 zqs+&M8D+BC3AL75bGb`>x*6piavmCt7c&lAQ8Xl!+t{Pxo*joS;r9rdlZMYDbY3?@ z-LMaV5ihSp+o(5WrKz*Tr&-Onyir+B7h&L=rv$UuN%Ogrp1vMcb2}cUxKsb^u6@t=gu zNAe2^0e-e9Q~p~^{?&2u4t!wtnVkIC#FPg;1A|a{W5kSomtPS|EoUs^F7~M>$|)ob zs*e{lwqKcGM##5Wiwb*o5?Y5>5)>v$EhaQxUxLc8KLGt--hd`iJ!HwL`sVyn^{pMK zB&Q#s$D0;{Zmil8?s~OcP$jp&g)y!V36Ae&EeLAy)~M&YkudMxuQ1f(BiZVc(NZ3i zPYOUa805f<_g@hFyr)@mXFOcYD3O{1tBmUjiI2uDBe?smM(KEl~;f;n3~nEvOy!p4_cI*Mvj%C)C_+>=v0wiOSo};IV!^bFtmMH43(n#$<69Ujy#xk z>uXe?(@)Us&2xfutmZoIYP8%@DYwr-FW0{a`tKE-2>qw+P-VI?@Z`O_1ZR(7>j@g4 zOi>+Yf?m z7lP;Gr6|3?gRoCd%vgH4mmF(pAilO51!ODCOB z93?Sa3|V#l5==DyhfDWIs$00!@pDG?NQ)thFZ@A>Hv7Vp;PBWAr5YFk+vLQMxtBkX zLoDZQF@c+lYKZq&fOE3jNq3>kMt%bzfS?wyvOK+goXiY)WtKA(9fU*8sGm&mhhe*&VyAFV#dVtIndp>ouKn*+9B?G zc5X%$ON$w!FVqtp%oHOCs*hKr+IAm@xpHF0kjoN>36_>-D7mNmVEw-9FkMm1=zB$1 ze2-;hy#Ol@J|!fmOgh4y+jS?DwtYP8Zof|`Ix;GX5T!0=w7(S&Rh)`oYu9^n z&hfcNc`#UO7s}#x7L;{uBh>Xyjpi=x^hT7A?pbJfua+>lS^gNIVqhgI{>)M6YH$Z? z-5(@(Z8M4C!9oc!W90d((9ZZ3LHUvPaqe1ndZMzW#f(80o)G?zr1Oqy^3eY<$d({` zNstVX6$AtXBuMrk2?>f7G-^U{$GX=wxUHzcQE|tb5L`9XU9|>B)w*KUx@)Kj!BvA3 z>q7l~-t+4p&uO2h-}k-ee9k?$2-#c~)aTZ@ z;Q84{!PVi!j8(t%smGh>;_*-&t{vdfc}|3dvGJ>LCplaR6)A53A3nQ=uxCl&YJ_PC z#EiwiEdu=pp9F8b{D{1DMcx`ba8%M1sLazA@Z->@2xs5X*W#{z%}A&r>03eSA9oO* ztHORnIPzHrHGji$P@H`pA>8J%4xx246Kar1V#dL{{Wc&ZaJf*=TIYd>X8#7(h7&Uif63m62Zo9# zLOpQp26xPBN0<~#*@U~o!^5DwMlr$YG{`5ZZ$tiZmdYKP#XnZqS4I*%uK! zZC=|Du8b~$TJK5BIMVPdn413v`R$F2pYWy@?Kr4qe#DIBjShskV)Ay}J-RytYTwj# z;G7_0M(*zJMuZ@)0P1e*LeMn31)LC0%$WB}F7Z9qH_>FMORl}(x_M6!hR4zyahH5} z1k~%YE#Qe~w-J^u2{j|g6NnjOf2#vs73aX_m*0?Qtte{31KCNlpaM@H1aA+0fw28u z>JHqETQdqOC*1^oId4a}po-avu=evHs9_scgGt#}5MEyK-Gy*`GzV(2Co!YUa2$-z ze~0Y6kwtuuwO6Z!n&C&xSkmZ12q;e2gS(q|heBe;8ks&HaP`Y-v!_gEdGX;4R9zkmzp{fi)p?QOwbajp-F`tBwPFBNX43z_kEf1+XNSH`b+;Kc;Exk zOsE~M1K^~29SDW7slVdR>u?R!)iNWv@!10e^^)kL2O;Zq+0OFsbk=Qp`FNc-(?=hoA_%cKn17o0`H%1Lolm){f4`VpR1tqH>?Levu_|= zxk5UDuzs`%YNRJIW2)g4__p6?B}|aQo2r$m8Di zCBDbXS@S*Am$aY33+EpptW{CZ;cnPxB~;ReP2kI{TL{Om1fNG(JX!)J^CV`BHvA4c z`+Y^;d!z6VJaC3~22_b3F=IfZ8{sB1sMW3Y zVAbqP2=s8DO9&m!EU2R*9k|eS6qL++iyRr7c^PkdcxXJ-p0b_bv}aC);wAA{aOamm z%y{E?1Gu%~I>M-zfi{H96+)l#ov+lCMt?tc^EK{FR>hiE=H z$@LppIPX7Xuh_n~@W88wCPHm2+XJefwIif2q1?vZyI#bMr;cv~msi|G81%Bo9fbH5 zrBF{2e*h1hIs?uf`URQ)uHY^n7`$dW)cv&mp!xiBgo&!;d$`O0Tn**9;V1A)*53&0 zuSDKQ7&*EeD%F#i@$J@&2rd1*|3+ALqXep4I~R=eBW8SLdV{==+2;Y?H0SO(DDKo9 zU{DY-%hF(*ATvi1-2qwY8F7P6Dn1(ZK~f)Kle{t$PrUc`*e$G3p9D()b#Uxqpm0$0eOZYM4Wx1TzPFm9-a6Cvka z5!9E}v%m{!hrqSxUmy=trM2NrNuNhSyuN*}n*WOL{-U-Hl$v zj9ZUy14mWdN635`@h`&1sc&(2suwZi^5aHuP=y^K{$+G0!jt8L zpbjLi2Iro-jKCl2`wk)a9UJQY>RQm8b{w2|z7siLl|_7y<@tF6)RpzS!1Y-!gppTz zy~kbZ=%G+=J%|}ww%$Nk)-UJ-Lir65RGfAR_{f)-ai8fk@*HOVM?8>wcPdoS)L+26 zBw|L>uB1=6o4`>+<+U2XZ?ou zs0*tXfNRsN;IQ-W@sB8yR5>2pp57Gym!H0eJih*CaB{ZO`h|2uD7hhl3eYYCZ~78LZZmyF9>eVKh5wA%cV|F-o^k*@Pa=j~y(`5VcU7Dk zD0=I5uw&K(gri~6J_rk&2SQ0itH4OtC4`5yKE4QhVu&H99nyitWye6jXK#^jEY9@9 zo3{2Mh8%T#Czx5$hVb!4yg%;FF0X>BPh1ZUIdvU@I+PTE@Vrw9b!7EoaDLh;P<;Lq za=5BrcRaB5(-f#(>zlzTSY4{JDRy;+x{en)$-S*`pp~fX{1#?c_LHJS?7LRbD zQwFtm^>T1n+IfVe^B%nrUXEl!9se{7T)h4eD9d_@9DOA%0dI1S8V$AAgP3u~)>ed) zez6qX1>C5Bx;bGDxXqWCag522ijd9ZLVdn74?I8RH*htHn6YYCHVqG?b0$J{*mr|R zXSE|N45QF-CutrA6)D;XK5Y97VNY$3M1*NE#EivuZm1YIMS(rn!j2PiqkG4grE0HL1-OW z0=4VY|G+8he+7%O-XQy2$wKJ0i^h0&vJJ~t#)YEUs%NG}A;(>#D5i`agKL|b<^aA0)i_|RK&0RhU zil1l#gHPEJ?pMWRBbYk}K}}q}8q80-g5Y`Hw=cq#ksPS?pK8I8>yLw}S?`eFUd|%E z$7&g+g<9r8%viqFg%H;-As2U#t`CLUH(>)f$CsFqYYNIk2x5w%?%r7fnx>ovCyT zu+=|;Noju~ygV1cL^wW@54HHy0#LTz3PxvrKz3fvA->1jJ8BZt3=d+)lC6&s0{YPg z;O^%2;ZWNqYzD{p5;JC-LP`)m7fGSc-&qE(o^lqfA`vsvclBrCfgS9bP)F?tz=g9q z5F}x#Y}`dQ*FZfK8o@nn4-lr+MspB~V~8334y^)j$Sxyned@zS7`2!UmD!7!@#C>$ z;MqZ)$n`HWd3e*1q&&by<_Y>lQn@3HB(s&RvW^8?i@IEg|gu64>RZuG@7{I~4#Ec21a52KO zB01EdJ1fB2DSsdcN!>~jLU$EIJz&oPci0bulV-g_E(~K3-(z_-kAb=>G=m%4{y|XJ z_LSl-J%*U^-NCisDcMzor2cSWQrPuaJOh6l<7`AIAO{qgglZ@1;V$T zET~It9k|YZ6dXS5Epl>L=3u<(wPifi3E@s~X`2&4UK{@%?qXwz8C?e(z-HNXgjr7m zhaj*Q3!wsg5i{OCb`sn^=o9j|7kQOP+#cH;DwV<5!O~ER^e_~=SZle)my=r zsdo{Mp9>p`uy|w{lk?BqfnWzPo}+J6Hjv;ISl4C^}* z4}55u2(?GJ2b|W{j!;}nQQ^)nhM4ij!HwWn*-eB|PkX2lG8dOZeT*k&JbUa6SU>0s z@{kt=)p#Iv`E;n~^!?zGlg|<6S0!t3C+@6<3Sa#b*qZt`!me|XH3(BimO~YN(t|$h zFCtva^d5z<`Em)AX4G6T!-JUd{gyY#XY%@t#+z1N9|tvf!VWOOmzeRHkvs->hl(np zYVWKAg;TB}gpvZsB0ShBfZD-c1WvNIfQ7R@B7249jl%=4S|&qn6k0%a+Y^NJTKah0 zy^AJhJaupjxLkGzVbIgi2?+6vWl&Gzi5U+ZJBKiLkcSq5|Dp&gc=;^wKK&4AKKTN9 zVpZBiyeYqP6qM&`6L=-H6=D6knD22na%2Tm>ZjG<+lDI$Et!6k5SCr$K$VZ02gZ32 zGd|k#4tZZ*Ht{{yoafX{r084nqgmg26qNDU>t zV*o>^+(UQ}81Vzbj-7I-N$iziq5Tg8uUXxeAzTe5X546*1FD5b!1T6P$nSIv;(M%9 z(Zr0)51PS2vWE!qPkS!M-IGP%K^=%EW}JKMDgu8{zzT%m7d)u@bqhc<-3m@T`5rmH zDu?(U%d_))s4J^}2G^%PMi_aHs>faG$l*|LKW+kB8g3yh%M7VUD8DR$iW{{IeDv*i zaNm}%$aC`gufhYl*JnTlO*jDF^&w_78B+!&sZ>K;VU)2mQPJMz@A%hw{YiRC<%Kl7-_d5Je=vj6=6>(G2=AL ze6Uz}0`zP95BY|!uK^F-8cobN>fjzQQ}zhq;}gm@+?`#dgsP7xW*lf2vJrPaBQ;RhKK=x5ZulEP zlNn_~$hce%^}ePaJoD`$!pbe)jR=GDSWpSq=Yr419|aHj5Hr>q`<`}Jb6Ojg}VcbMncVvCuZayyMqusC~P;v{SFzFxo$Z)kxtB*f6`+Qf@c*I z>dM<$;QCdEz>%phkyFp5{fsxg9WffJ<)aB)*3gPjo*BCrcX5|1pdQt%0r!2of-q-` zp9O)N$At>IJ`cP*{x{I%L(DkAnB9yA<`qqZ`gVIacxiGw!n#1pFSr}Na~M=Idn5Sz z!Cwd`X7<>Jur!pIQEpiZ#tP4XE@u~Vv#x;n9&1)KF(donJ}^-B9O3qp-Uo2EeNi>k zxOig5oMZP9z6^{!h;X4p0kyVH4-TUfGbWw%I)w1DvIOe*+y8-!SN#ggQr{p)pUXIm zH#tX)gWCJC37pa3Kq$#1AHiL~95wi0HP@k_Y1kaCe0ayDF zGgcXMkK%#!qRCJlxA%faCqF@07)U>cJIT%wP?78{;KK*E5%$asJ&rIfl$f#DQV048 z&w)3b-;lTJihjcbM@17eW*$5Uew4mIIQt~^1n%k=je;5yPs~U?W=D8FFvf~-q+<}& z{JPbkm`=)Hq{)aH-7Kuv=SAtN*4 z6z<+%8VYr$W&^nL+YN-lTY^p_B;<*qo?Tl49vXidto0#g6dLo1@3BIQra?Wp{R_Bb z^1ld^0(<|CyTYAnC@=Ol@altm2peZcoJCNF5;LY-R)X&Y7Z6T4yPZQ=t{VU~D4Lit z{@@YtiS#w{fhP>&d#t&O#zOJqi5Y{BJw&)aFzyfBnLECNnppQEm`^8W^gJ1G0pUs| zA8P&E1>ne4Rxma71M=ImImGu^Eh8pDE&KR0Sl;j$Auf}433rb!4TsuSvl*Q8?G^%e zOUPw}pgbwm-D}H0)A+OC1Rr9?JY)YWc;L7GGodcsJ^-$p+<`DWFx7^;EE z<-`*+emUBPaA9Dt>$qFnQ3W-uZatVpCuV%vLb`!)yix?U`0Zj)w(1lZo%$KsdA8qQ zc;Mab8 zZnqKW#$u?B{&T>iw-190C%;0L1Tya6O_4jtKs{uc!95TDL6|nP=Uv^8A$ybck?@x zP~y5xU^tzav9%@m0m80I3DlIgKY&H6eg}P0zv7>PUprf9=k}B`{5Ox70jc@8AIxZQ z}G>z)a$cEU%s8p{}xaf*T(=5!5r|ow!R6C5C*rw*fpQxQ?*g8Q6v}NGF7f zk0yqEa^NI*K>7)J?vp$h9>`xb1u8h681nwnrwHbOiS4+X*fA0+ziun&NhgN9(h~Lv zVSQy8)X2BX!PHgf5#FYFK1OIc%Y<4sVm4U*@eml-@Dlk^pY$hq)4ofiq2|;yg4}Pd z2tiw7pW^Or?qDd>wKd>`@iv4!AOB|v-?nq1F7=-euDkskIDGPd$jO0y|HT7e@0bX6 zg0%-+`k);_K9lkscd? zoJG^2zL1F-FC2Z2uy$bb3)~IssD?_a`w4tWBW66_68REgab-D_?5!S*UUd<{nd1El zVei=zs2L;Xf+Zh+1p^x1Am8lM=QZB6?b0}?F*Q5D>~9W)&zs3_aCbhp5^D9ebzs%_ zYY23oz_$n;+XYZZ`!50)-fjUUlRqLy2Ih6*fe&{~hT6ljfYTm4K`5R{e}_B2P-4a# zd$)jF1$PieIYa+L$kfT8K1LBUo;`35pSUrG2@Y=FOcUCO#6U0 zi91F?h1Z$DRvIzmu9ldOxSLX00af&NHR!YI3c|G%zfTC8&vKwNBj$k_ACH6Y*S$kN z(W*np8(I56^@D#A(r5PmhP!tm#Ehr*ZUdJK?&H6~gPai_yp96@ zc%1_7Nffc;fddzCGFR&5i4*>l0Z_q<{s-PC6FZuZzDAxnFryn@mESQI%CoKsyh0;( zT;CGsg}afJL!eUM{s_LU{}Z7lrMox6va@`s@(~NcxQ|xw(Yg=F`}*V(A7stBGzp4Z zvlk5d_88&rW|}YVOt~YVCS2PL=8eCN@Xb5a58={wDb%|Db>Q&ZXTjvj-;iH-FY?C& zPwbcpwUl)blt1V|h@F`lfIC+Rv19XIBRET7M_@anyCVeZ20`79B6i$<;4;EEsc#@c z&J#A&mxZ<91v0VY+M}Jw!v1XVI(2^jG46nJypXXI^t@`LffF_)%7W!L-ye*XFl;r!;L5ZtZKRYO%> zGl2B*_YgX~BSI05ZkIzX?7tF}-2MY0a&osYgooXU8Tag%15RTd0gE5JLiU@t z5kkzkb*~v5C3uLC>FgPSyN`3fgE|{U%vgWmD#8$HKqLb72@mS|!Uf})5d1asev3TI%OEaJnY7T(UzPb?(ZBFTlyV~3uDB-p3VCeV<2oJoY;}CXi9|$$6 z|0=NX_9X$VSu8s2{`n0(uY@OqMeZ-f)wi5Zvf zm=DTXC&1VT{~^0(^i9SCn?r~hXYJhsvIUP20-cl;+})n5gxVfO%sB4AO@th2a4N!= z$E8pg7XARPB@;6aJNgAVX<%U*9{AEd9qRb9{orC6F{7*{IURS=l^Q7Lo1ehF^?xJG zNQq(~l$l7UV%AB}@4-jp8#D5<@xZMi z#Ehf%TEI-fQ-qHWdSBd~ojVe$K8l!e$bmZuRB2ca!t=*6s3QxPgY(J6jN+pnxd`C{ znNY3nS>Uc^hrlT`V#cDDv^>1Yr*bsZwKpbkbA2mqyE(f64;1E3gbKa38+n)N25O=fD!EEAIb+>zDlsj-(MYrnY31 z;7xCbjDu=<(*!Q7cOaCfkXg8kJ39pG(eQQPz7KyQ%vsl+jlk_AfC{>_5WK5t0Zm^& zAWztw%fSQlawkK5`*SaNY1|Wpb>4I??uKt40hQc;3;6ohZG;n(LU{;FyAw0YchrHg ztaG62?{CP>Gm7|l;H(g0M)ux=V4&ay!fi*Y0C(Hxj)EE&Ma-CUz>e@m5+g*o@OTi^ z+J&pZVPs;)q@%tfgqP(UsN?QhaPhL^po~V$7~PU3#+#f&v`~BB>;h-hyAVoJ5=wCw zaCRuv&EXrsZ69tRj9C{XLCEeShWdPQ33y&}8eIMLGji4DeBy&FdhRr+jz50^kB<8n zVWD?#8SW(8)liZBw}B6D-9y+jDWVKvT6bc`;vFkNKh_0=8-I6`BW#^90BTeSF=OW5 zBj88=Yvi*I2Jt~w{oJuoL!yWosRte+JeS0k1`I@KE$2h+ zaxVa4MHX)E{~aC}nmZHf!Jh}f9pgF>CV8h0!Cm3@8Yr*+M)2yb z2M8M{MOPxIyAv~}?^p%CE4hqt>TjPagyl2XP=i8<8RPdJ1E26akqfnzu|ab z%aEy1%ic7D<@L`H;!=`E;O^1yDyV(K4d9#)cM-Vj!bc(m^^rr}y|@B2Y5qW%@YPd= zkhi%Q>RZkn@Y0`$!FA(aArJRvsPU%c?PH)`7n;EnxBfv`I;m$h?&RHx8Dn>>1zjaq z5t{$@*C5QA!GmIl5HkkuJptb4zenEg=v#vaj+^^ER8ABz<3Hg-3S3GDPwVW z^LGu@w&B~sF(3YpB&B4piyFuK|DQ}3=@I#6ZMVp;Yr8QbjRwz1??~TBe{u|oHGkgL zAm;!7IryW63nrEf`2RnO^1yE&>pVMW^StqUIP&Z<--G2r@gRA4vpiKEWDkE5@=RsllQ@ZIr5?KMNZU&DOR(E4J ziwA=h=;&tk$YPNwUN(L)<;Oi(tPqMf$&<$lRe6&=MXWHR zH^WoP3U_!jJr%47ijUN@f)%OqQF>OgqKrOTPbDkb;iL0Zv0^B`de0hGtjgEmIhNJa z=xg!Rvf>=RR?o>SGR4p4Ih_@+^0Rx+X7w`qxjl8P1czUj=K>am;!o#ST#kksoID^C@q^t#9DXAIJM*;)CHAf1SPtGg6&=(SObi~Zm%v@i6gkn%Y)6LgpjBjS&nVDSMD3g6X4RS5P9QJ{9c2sz{|zCHp&Lq}E5t9^#19`KZ{HlqkJV z4ZBJeW$+ox9%_uT_-NU~98p%E$!sMh+U7HzJzN!S_nFNeVT^YB=-4A2(Oo_Z*eXg4 z$#)4`t%@Q0*0HONF$`ZlTjPjf`mSNuP-3ON4eU{>Sf%eK_Gn|Q*4Mxu~X4|2H)N6@y4DOUkiJJqo>t(KU+(Qv-uuoPgKR(eUGufH^#Yrt?WsTxGvw* z?8y`|$?qI{ii%A3yU3nuBs2VM>}d`%)9*TaIwfA}cZ)qk6|eNW$DV17*ZSGnvmEg{ zKL>j@rI+5Xojpg@%i#Bv{Xb(bi=Uf4*U`)B_lm8fB-s2q*|n+!yWji|?0Lonw_g`~ zz9XT_&x5moLLvEka~7&7WPg9oA|r+2PvR_gP?-K9oFx>h)IWl=R7F+#$8dfyQnmhM z&N2s8=TG6(QD}PqM9y**&ETKHSz)AE{282;4w}_Ji=(H|ZT`8OdKKO7U%**qq`Uo@ zoYf9`mp_ZMhLVV1yn?ejP>yn^nCH0b@B^jJ+)ZTFzESZ)?D0j)9VF3z*K?rb@O4%;x-L zOm+w8INKe`T>%R?MoJ2)`x1^xl|t@b$7wXCFuLnGW=9IM`x;IYB~{wJfwM!Es_edr zv(uQW?QY=ga-{0I8#%ivY5ML>oIR>EL-*aBpN(mj?iS8oN1CA!NMg}9$#yR9*Faxi14pTCufwwqE zR2j;^dz@d58QMTQ=cpq?7wF&|qx8`SwsVfF`WOPAa(*-Ru>`s~CmemOfv-4LN~SHa zlhdNgvoff3(gzvY4b0?m0@f zloY`|ugX@EVz_@8v$Z5L_ktr^N1|{qQu^viiQG%7z6MeX_p-6Cg~Z@qarCv4vbZ)% zj*XPdy{gKwlM1+h8gtwvCij{nr;EhmUZ>=ef_U5;s$6oAi2Ii@mk}i8-gM+LgB09b zlssur1^2cpPZ?Cny<^PN1}V9B9eKJS755&cpFXIDdtcSh5Hyzix3QlkNXvcT=w}U@ z%(YYUZ9&tyt*U%`&}{BM#(Z~>j{DG&-xaig>!1{ndMx2ORR!c8b=)>%0i%bW>v9w@ zd#vHMQwpU$8n};Cg~}eAxQ~s6+8ze(6Gx%0hmre~(qG@BiTg~|-_T<>_g`awOAib8 zxud_e$9}GxQe^9KnA@Q$viCT~ePJwe_powbI*Ph_oaVlwFiFAZxUW@Aa_~j&8zYku zY~#LlFqy&Ext)|^Y49!XJ5{kV_#XE^W3e{a&VBDF)&)DbA1DL#!R_3SssV=Jr`%7* z0hVAl_p@VwHF*9jZWpD*7Tn4GqAIZmf8c&KmbinvxZfNlUBMna4=Rfk;?46^v&bR- zyly5IBZS2Bab+Cp-9L29l&q=47M#C3--dBIL@R|tz2LgkS{dAv|Hj~puE zg_(GaP$@6m$zz5pco9^-G_-;jspcy~D|t~SzBW|Ji+1vLp(8;l-*2hS0IR zo+g1MRLhHV3ap`%d1R{47CN05uNK-vXY+cQgziusFTpA73SGdXP(`G$B|NHHL=LOt z(M%#nn4U*>ikM+*c!^Z8G^~M_q!uf~Ht~9!#M&?eFWD*9g&BD%)KYy|6E9U=Y6#oS zOEZ;P!YsUWXQ?%8KaW9`*uoC;GSm`#*fCxolf)fnGW;H|pGl?-xAXFyGF`ZXS3oV(hqvXl@T#~fvH>@LFNmc<+=z8Uql_Kk4WT;)dLL? zDg08?KuZLJFL4gEMr84&)Iqk0T)s>_$R1I^FEb5tM=<$v=b)|#7GFWFAVu={HT){| zcZSHZ{Gq1rERkCNFz0vH$jN*qb%-r;I)Au&h&^&Pe}rj>J5t9V=^WA(xqz>tR+6HY z@YU){a#S6^+EmGi((^UWN@mm=ehsxs8r8rbrLIy&ZQ_qMRcWIP{4vfdU6heOmO4}) z)x;mC9%_i%%^z%qxSQ))M2)$!~BWrVfLtF{O?V}+)-BkBJ^BNGo@t~zx{E*G zIkGF-L$H9VBE@(M7OGX`7=OVclZp{T5-fJAm@y%OB~-OECPJ`Otyadw2!1fBwJ~JD zGN)P>LlM+btMxI7g5~OJLrjWbg{j&S!w{@=R$F7T1bV8*7LzNeS8MDs1%g#3jXQ=Z zSnbqw#jpfxs5PWmo?xxIh8!yr{Aj9S#7YJ0oHfi?g`j~tN*Y@sSg#(XjI9)GFpbj2 zDg_&zqja$~fCN^)w20Q^)IjHVO8q#~XU?7W`})Z|P|f>~)T}_S`S9P$$@W9u_pK zC)j%)6Z~SD;O=P^>~l`&>UmnQpQ$f;$5OGll_Iin88|vxgc#-fg({x6>RCv=l zof)qX-lEQs##ab$t7j%vZItu*15@SVC=nQ%||pQ%=xU>Cl3*6I@GJA@yo^YjVr z!jI~ChJ>fWPo{a61h??BbDlNfm9UFC-@vWT2#ox;HMeP#B`XwnbJ-mWV`KY@_6gf~ptWDFvb)jf>qBrYN{=aTkRp z3ZX3_QF)@!>Lp~VNEFt%gh7>x!rPWGsR~g9ZK;%6A&RVCs-#wmq8gWKsY+3F+fp4> zC5oZ_pr_V|Vyk~JP{)dTHvV9tYDIBvKUk@gMP%AC8+E!UzIvIRI$PAMahaQ{6D71Q z>!L0YQD}7}+7c19x{ge%6VV#$7&N_z-d4w?tq~>CmP=_3qNM8OO4=q-@5bd?nn9G@ zwp>Ruic)AR^t2{XYV`^OZMP_`afO9u5v8}Su+sL67_^l(+F?;f^-4SKn5a+VN;l0a z%4}QNMLR9ZqUlNWbE52OJ(+${)VER3pxZ<_ZF(mCx+s@cFQwlSGwqa8tb)m zyC}b{UPpI`3TUhJ^mb8U^(q7Xsi=SBDhu5$Dr#G0rN0s}X{&AYPEm37YCHXdXh7p? zH@!<#(zd#b?jdH;){qjt#q8=eZczk01QF-9zC zT&qnai-m1#b%_+Qi1wpCF;Oh8{?U+_A}($G(UQmzOWJ<4CT5AHv~{+`T(PWrojtKY zT-LbGoyZi++tzg@vcw8n11X6oF0XDNCyB%Z8ygr&Qt_a+24<2%TtQneO{x$Nu3oQ9 zsuX|MxL%v26c1@zuS-&iD`^|_Nj2iC>J5gZvEreP8!Sm$@vycH)}+Z|C2gZEX}WlL z^+tQrZ1IT3jqW6!cx2nguA~KG6>Sr#_Y$$XdK0;Kow&Mj6Qj3YtZCcC?7c=@L)$Fv z-5?%Sy;<3NlX!IFW^HeScud=7U2mg!ENzRvcawNr^%g_#-Qw|$TP(dT;t6eAtiAV( zwY06a-iO5#tGC*F9}|DyxYga;DxTD~wX64O@no8TlzdJ+rP@GFz9^pBXkaAU#M9ag z%;f9h>9lRqPACK zO6iKmMoS8#bY)wkH6^Q5Pcz$6a!c#0&GwXn(p8OScM7v~b(^^>g;lzS)m9DLB zBBzQ#*}9%b65(w&Wav}uOYU2S`G zX~xpsw4e29O{IIPe>SA;F8#UjXG@x;bZ^_w*0lYl7TR80+TqgX>b>@~W2L_|?scbG zOZT9m0D^0Z0Vh)E!F$% z=^si@Htut$ca@%M+t-!uAvsOkPhxmW&Q$LwGw{Pl8uv38B+1#f{Y*xP`r!ctFb_OD?n>&@m{Ii?oA!Mxx|W^+5w8MRK|EpoPJZTxmOKWn@Wgv_m#V zuHc$@@L~AH-jm;)^@0i!IE639VTV)BsZ!LlQTq;zZwrSGNh85ZHJi|3dt?n z5otz+a-TX$TjMcCAHBrY zc8u9)jijA+T-v8W@~HZ_vd<>Tl4sSw8T#y&{M-1O zrH@7OyzMt@pZyXy?S!q*VM#~z345Pok{699+ny$ zk~fW3My5^jw#~}SoPS-?No$d2-jcklZc%35ll<4%qRq5R-nX^rG98i+w3GVGcFD); zlZMQvl245%Etzh~=eCp9%vX{w+9_LRr{qiZDSPGz$=AkH?#wR9x3*JVnI2LP`e{;@ zx71T}nw;e??PfmB$RbI-T&I~?AyRMp8EIC8)JJnhnH3}THJ{ODk)?jFGrBB_)Sv#l zJ}Xfgp!wa9l_KqK{@s$rkOsPbw`OHY@q;mKS-H|6%~^X^fwYJDtUHS-4R)RF%3?`F z=;uh;JZY%r964Jg4Ktr(WJ{&tu5-+6g*1YGUYcDYjntf1W>-q1%;&Y)N@=v~ye?ZM zjiLXc&#sZiYW^@}kCpZ`|6$42O5{e*Y-um^1$VYin&7(7 zmAybpp$0`)ekp@~#n$(*G(&U6-uIZakNJwb zuT`4qy3*D6v^0xuBjucvW@~KZoQu-FW*Z~NCe3l#m^s&_x%8{joLka7%~fU2J!wDl zRc(%4n(w-*%W+5x=zr>S+NFh>KMgrgrTxu+T5{adBG;eRoL5pN{hBSOQ(COKX3zN` z9bmrZ&gqhtxUO~Oc*t1v>!e(78C!Fmoa-;+n6EQ(NiweMIx{y!#-rbm=0?c)nj6a8 z7@5F)Lz_#M30*gIxfGd*{+B*CQ6|>>WynpDm74#ui;=9}(ZrcCa-*_F$ZDd@LIc|2LU<`y|mBpYbH#mJM&2DxrA^AxfQ`fX`mg>10q zwlc3$_MQ2*Hcu%V;<~NNQ^_jncl3ERvMS9TL*7`~Q1cy2o>n%@b;p`FS*E1lwdGBh z4cFYY=gpRlFyD3O>0~2acf0Zy$W-)uq<%|eYRx@zzdBj9`5vR6UZ!!~WAhc}3+4O((`R%efntu%WPi6lz|6|E_ z%jUZNvF5*$>F5t_`JJ*_%|m~}2E~l=5QdUQA(-$O`E!VUe3R22e znAcoKav(!l&#l1QWjR0Z7@I57Ang&x*q8YRb`v#kM)H$Wt%mR z4TWRNwwNDV3bkchU5~AWlgkYBC$_@rW!p4Q?1i(-elkCC7wXEkyPk9vE+{k7pOX47 zDKlxFlKa<{HJYC?`s>TguBXiYYs#AF&!qhu%64d;Df@3K+i8BL?QbaC<$9*;Z!FtQ z|5x9?scettUqk=hWj~w$we+`??REWY?Z3awLVs@Sf4Hn!^W5J5SlKV;=kEU2vVE@S zUHwm&?Weo(f0{2lpmCFnE|wiMyBS5cvO_L6v*>!+VS0zO=vLVgO^34RUfHka4sDUW z?5L|lSL7%=Mt`9%YA-vkd0{AeTK1dyg{8<{cEa_-TJ);SN`Gl9>MU!~ytEg6C_8C> z=`QLjJLP)WRpcQ*O@BpVddtsfUXk%5z<)QtVlYYav#wW6W{CV8{k4=CAwREqtz^c? z|1iJSGRg7_uGcyyMShX~M$b%?U(&oWFjM50&2KDBhWv`_jg^@tx6$9)n7Q(+nzwdl zf&5SNTQ`#_zvg<|#bn8^(>qDUJoybxC%IT8|I6ITD3;1^x;mM~3i&PiJ85x+{I=$u zvba)y$NWxPtd!q%z0(z|5FUR_ci|+ipR?THveZS*2*8a{<9WOmfPv?ZN<~& zt(y1t;@R?l%x^zatHkbX}}V>Q}cm5pibUq{=gWZm%Cgam;=_x+vy*r z0~+LyG#`}%Hpw5GKWYaUHb@)$UQu3Q>3`ddgT43ZEL!;j9>iZ60X5yWu~HP> zo4Pf#7>dC5ZY``V1u4<%Dl1nJRO8jkDp2%j^6FqQ6~XOZUsx)ADm*c#t$ z?6HcTO}@=+ts<`7w}m}fK~D6$%AT%>ukmYT&sOwm^6Oyh6bbEqU)T#2ltlj^&JqQ+ z#y_4@r=T_YXK?fidb@uyXN@8;F+j#?P$bm^4Cic8^ll26$T29A+XHGjMny_u_j*o~ zBDJRbHqLHET2uFCjzy8)-o1shU%^NWyvjMO$fyZydr{H1iIl;$DRSCL#oX(P+{7Rm_m(2BCTKYKo}yn<&_u3Xk>4Iv%XKIU z5_{Bh+ZBa1J+^V5D*89|Xy&>VMeRLWxUUq<#NeykPDOD|a4YwNVn9=H2e(U6(jNSU z>ru{13<=_Sm$Pd^;(7k%oF@Fz^`vrcdq^=aq@0%+D&s|z^J_wf^J2;cO`#Kc8Y32)^Ul$SMyckr0y z^7imAJXX0PF(Qc1D=)8!i06yS2R21y@TKL0+9Qhjit>uYNEyGPd~i+VaDHX^cTJHK z`O5Mk?UA*7Re5D%R6W0@ys9Q@8-Hy1(59$nzP5Z=dsGX5a=9`w`YL~V`S6|KDa&=8iyr8bUx+x|@pfA_7#}o_Jl-DH2$^;GNqiSM@ z3pSOHZi<~KFqDsJkF6CL%f}}EUm@rI*Hi)je<~^}MpRf>jL3E^MyQx*jIci zXWKbvw{ymogs0>h1Q5mA5qB5e|wHQ%RQ8A*TqGCivMa76}*J4CPMaAgb=kxe} zfBydZy1ZZSzu!_%--0j*sP*LT(+r4+FLQA>Fvv;^chu-7e+cHw2 zx7-uiJ6cigyC=GLyrPDZLMoW3s6|uA1(OwZmK0XOR7Jfngbu8 ztJr5D>k8&8_WQ`jg2jqPN~)>AS8)JMH5Y^`4q8%O1xpo&e5vk&m5L_Hy`F;A3L|>2 zw_vTJ*>Z2BV7;QncW<;HhSy3-BNfK-4x?%0!Z_X$OB$;%o_Exj#x6|cwNdWl6ejVG zq4&Xs$-Lv1`*ejVyc53rjD=~ucFGo0A%%Am-C{0e@J?B_xC+yGr+r)8g_*n#%Ke_g zES?Fy-&>f?>$Kb-Da_%W@!cOS%;R-YD5Ro%-dU7FE-K`mvrt$?rM&Y#3cIMB*G-{v ziU8gPlnNJtyo(m9t_b2?@==XNa$XOGW-3B?mr*<@Hk-q~bQ-Qz(O6+|GO2!eAA5@SgE8*u`Bu3x&xk z?&euhCS2UZvsswB;$EKJ$21oA^Bj~1OvM(S6Mev3?BG3XdB9aXzB$$Qa~&MKMWz2r-0m(1{n zDJ)LOEbnEM1((e6Ua_!rCG)&jeJo?iB5#C}VJh+QUPCj?B_ZDHmJC>A6mA#O4Iot z`L?-BGx<}L?Vi#sz6agzEzRanTee3^bNCDx0r;PzJ75mT|H86E$AS1?`gRyOa{e6UVG{@Ce}z75<`Dd^Ef2dm zTK+e_huxeS{yb%;hhyM-(VbpSBY(lNGs0=&f9u;B<+Sn_DcK}$8~-~ro6K$Je{acV zaXa`w__EpDF20Yli^J{a`_Wx6w}&6F?9y?2`9a?N6`F6XxEkpbz$|D}`Fh7Dm;^mI;mo1M(xTE}^eUC)B_U_^T8)r)-x0JIaR3=JI&m7C^!t_m*b^3D(CWP)`c|amHpHz_# z+=k_oD++*;5>RwX? z3T(#qnkxw4ZtGrGg%-HSzt>$+1Ef$3JQW6jj1_n*8i7=6L8PJyxYu70t!M?(sD&h6 z8*m?1NanQzTdajFUI%c$zmU!A0w~lX4zC-aVnr~o2cTJtbi7`G?k_U(`T+*D*u=8{ zOsv?PfcQ%gL&VSt5|czGj0hP5QZ8wDQnmqdBvKqj@6#Ge4R zVx?sMB(Tj|%HmG}+x?|%{tS>s<#713zz&Q9^XGtvtsEVH9@y#U82O7pHkE7Q`+!{- z*US$AyRBRoe+hWR&vo-xfE;R>hrbH2u`(}z4al{YMfmH$qyDleKSq#8Ehho7f<0I{ z8Hf`+W-Vs{|FM$&6k$9w zzz`H$c`hJbP~zvgflNUumG1$v1RRX-1+oQPD?b9{2+I8YD3B*8rvfBFzMujF$bv!v z&kC>vr2@VmU<=9x09C*d00IFffCZpHXcg!LkU-=Y7zJ_xNEMm{s6dPf%>qIou?k%R ztw8D*x&<`?h$`|33<4Mvc?FFE#43singlYxC@N?b$f+Pn*d|b5AX(ThP+CEjutT8o zgKS}!0HumK!fpYEiD6-n0Jn;D!d`*eFE$GM1q4-M5?TZrOkx(cI0TheiAy*jsPaqP z!Xbf{D)k751v*UX6^;n>R%t{yDya5Lqr!1P4HY7ZCIqz@L>5g7>Z}k;G$p9_Lu}EE zz(9pLqFF%$2E(E`!9FXj6U__u`(dMKQP4<5Od_A)0EU=FA;Cc_;u0+h4*3zcXhqOO zm3c&~0wX5#iq-_pR#`-}E@<(~qM{gKD^*ScV}*w?IT?%-96&? z2TT$k!xS)>EIe*i=)e@=3BSS!rU~1rN)t#Cp2U=9kRd!}Rl2}*;c36p4Q2{Es45Sb zB{X3wFPJUtw5lRtj_{0M6$SHzT~w4L&KI7=P_npCc+QHl#HGUXev~aP7j{!Iju;SL zz%W=03NKnQofr~c@?%D^T-Za!O=47d8N76omiz=;t)P-t#nBSgwOda-I5{U0JX{^85X**Dz9Wj zIB2bkNJfRv`>Udoap4eEOOj3qU%<3v>7?*QtCl65629cuvZXV^VXBTJofW=}>0s%c z@D;00C!H6*>em^ii^37A-X!%2U&HifX-N3GRqv863E%MR-O?4|D7D%nT@|{qYOi!n zIA*PmNY{mL`m3YT7|}Sjh6Kfm-ok3gP@L#(YYhvE7ro=JVMB?c32H3|N)o+`)xuD+ z=sjz#4oVTd@2@pNX`)GLoe83dKEUeC5JU8#wax{li$3z#xuHza6t&(1Wr;jky%)+B zO$$he|;3n6U|T!BsgF631%R}g`!Wb1{Pc@`pj=&!{wq`Y6AxbM4w{~Fbs;m zur}ymNc5$@!3fJmbJTq%7!`em?K8uK=xggf7pxV1C6 z9?&7ZqM-kP5ozfcg{TKjh(#2}4w?~%=tt{87cwCF$$!v|42hPghdjuzD1sgGA|s+@ z>!AoTD*D-fD2j}WR;W!R*@Wm9tcfg}6#Z&#V#%gNzxkWkvKi4T)yR>}ihjq8uxw8B zht;T)&5Qo@8;!C>(HgbcB=d>>!kW#pkmzq~vrD!l`p4hwmaT}^sVyGaswj%Jcx7v% z4Qoq8wl4bD-x8I@fHAaIk~|i?25%+HP%f&lNf9d{|T;5~ukZbc24LObD67(g<9!mDTm zQ*9?AiYD;hz=^1$6-=YGlay`XeRw-r*$!^8wX>8R;QfJiwz3PP&`xrc-5?b|2`hU* zn(d@c*$dJGCymN}kU=|TQd&SJe#)$LfDhPCxs(IogMm|SpgDm{CS2+S^ z*iJ{3qu@h<(^2I(m`Up(sV2a!cn4WE32w7>uvAmv_CN<)H3MeROdQoLxC1xAsyXms zn@Ojd2X_WcM%5yiP3ttNeBds;)2s@CyKS8=)e`tfpwq2d0dr_)JgQZYji2$V*1%lb znTTo~d^B(-s)`Zk(Yi=ztauOJMMnQ)FWS0TXuSCGKo=WL6z9{kS0Arw7XuF`p zAh9TL!HCJlAnl?FL&akJq8TH^65B-=rWH#A7u{Hm7@}SBUHh-X%t9mpOR17{f2acuS8Mw_VoZy<&CX zvJvkW6SONP+#=TCSIoFWTxq-F!Ux1vfh%r&NUWt@_29!|9e&k|kBIfQs}X!uTphR? z#mB`pv|f^WLR^dYlGT&qI$JMGJteLW^s?16Vgt?0QO}ASa5Jo)6YsN`b?SNX{(#x2 zUKBUd`b=t{_yFE#R)@p~ZGA5FlK4=d&#hh&H_@K-s8_{C{7J8RP26mIGNN7=w*;Py zs$(Rrw0;s1D>;n!lZiOV5nDfth?g7<^s|XXNgM4c4v{1|hCc-p$&%x?r*uS$VD3X)-(`JGpIc0m=MWjnk2cC8lnUW6LGae#KV#1&C64{bY+cOa&M{*|c zOq9rzbkQs%O}^wTZXs(5CFg7wmZns4K44*M$|c=2D@OxJF5p&J14=I1tU3)OxfHM( zHF8N0&1TY|lFPWwtRW;5x2Yd)8GsAbBqEth;hZGC+IIQ#mYg;m>(1M)r8~)e1KdvDS6R0z^a;(yc8H^iQ?)8_R1rb#DhFPgLz=?D0WW-UYdq3uPNHeLEr;6=AKQ#wU^$)nAZdhnOL+HC2x z?WKq|NBVK#rKmPfIzt;K>GGwY;KO8Hq4ZPRFiTe|{VXud)|E?VX)kkhfb?_xWmpGF zzp%Zm(?QZN11}qOa_JoH6_XB?euckc))CULZLhd=TIn}|SKPW9={)UKkIo?V;;(vj zjnW0%s}Wt3^xMFzQC+Kakv2low@JUlN67ki>G!q~mcB#!LtuohZ|RcyXs>bf-BLgP z8m#Y;25hhC^u5wx;5DPZUmBvlZqi$%Vf=No-XZ94j?R`rzhx4Q!kJf74sNCf%^T8L3{E{u_8RS{(z$(8ozNvCuW@adJ%@bgg}yRTB?g7aV8TBto(D zw>ULP(DmxK;F@IU2K!sOniS~9;9JI;G$@Y#wyA~!-K2ioT*H8Fw!iJFNr!F;zU{8b zgyQM%cxtjBlKLHQO*WKZeYCt8ySCDJEIwfWF(>IrgfA#}Taf>m1z-4UE% z*Oo&`^mjS60JKT{E?f&jciP|8)k4r+!FP?dawwVpo~agvHmlz=*Amd(_V--1TIinO zd+ypAD24vMr`7JQCz4(I{mPbeTq~+0c}-Jk?SX+ZT2Zv z{S>r4IK{4?fwJfxPW>#jL+yd<=b(q}9$o!Bv@_^2)-OWY^l4MQ589=kHrI!s-S%l$ z{Sx#@aN1qJ0_D&@_SCOJZ1u<9`ZXxm{&A#!9eOnQakM@L&ZEze46*PY^$gh%2R~+? zVHx7#$AdF$Ln53{|Ab>mf}c=-0vnRyz4lLZh7`CU_=(Yw1{cylH5n*yk@{1!fdLoW zKXn<>;gaB|ZbK$qO8?Ab$bvcQ&%B0gm}~zmV#tBZf}ceVd2l&>mei0BSEy&n4TUhz zKFew-h55l*c0)M~&_Cxi0I)#)IotrkLi^{s1_%}fKQ}hWVUYfXsR4z>>MzU<1T3+C z;cC#r(%={Fh8h^6f9Yv3z_9vDZ$l%D*uRW4G{Lgqm(hk+_&?-3X2 zlW?7Vp0$4pt`E+$_s_rvx|g$m7H&{`;r(;)KD$@9e;(c+^cwds!j1F=(|#X(K)qnz zAA%3s7hL<7;6uR$_x=^QiThCDGr=FCjd@5H-A6i* zkDOKe$Oj6Mb9Nu=Kq+!Q=wlx!N4n{L&H(_qp!UNDK;)v`uR8!Cmx6xd0Xfn`510<1 z$YpiFe1Jf%*aNNuTI6al;66};^wNW#0|vyb4tftXB7OE?ATX9>iu3>kjrJ_F&j}upe>Ie>5Gm zAWrp<=7SF8S^JN!g9FHO!5`fRhmZmKPo9Irh)eyG_uvRJX#Xj4a1?ny_*3-YI5I?E zA|0AQUQjQQ4^1L3+Lu^|rjVC{OYB24$S^&^IW&vBtd78k=8#wH5#6DAb`H(CPNMsV4EXayOi|Li%min!H3dk?K4WA>jTht`odgFi

aKcmK$R@owbjnv#G`%b+^Zf=)-Z(n0IcgTJSuCbfDWIp;|oaSzsU;P){ z+#?Iv|I#)0%7VeajLrSB5dCjcvqctG|7~t|$bPi{?P?y7{S^G$-8>{)qW|M*9+pMa z|9G26WXtw{BF&?+pM(EIo5y7<^mS6pgzOjfI=N+1_N#rJ)iNdfEx68ZnUSs1qnwsm z+3)Ho+%hNo!yeVO%**}^{`cpiY>mEQYVpbbQg4`BLbAW@8?Kfm*+0Pzcgu=wo&K+< zWmOhc|Lbj8lWo}ljkK)G{tf;cZHbY`Fk%u~W98QnF{!O_@@pM28Ljd1>q0TPt%>qj z#x>m5B>DBkHAri+{07H0`qmWrjiGCrThrumjB7hvDe{|$Yx`Om@|zvk4z{MtZwXyH z)|x4gXIwYknk6R@*DbVW%M%>eEw|>#Zw+0y(V8buWW**M&X?at#HJoDl;7@%%{W{t zzatczd$?Sl#JHY&7?5uwu15}o@;e>Z>kmWnyF%ADAC}9L88>tuM&+A{8~P3t^1B^3 z3?A0X?+M*7cDP2K!nkqzut82HZd^FrC{K0VxO})tesAc;jl-?-G)7#)kv92#L|p2T zcKH@ZT*i?O`Te1|+#_9b3gag3k#0GaxCuGZBd0lT(jV!S(?d5kAL*Af7&mtwvB;Uk z&3#84@&_C@4;~qiKNz}s?8uNjopH7}d^6jDc+@mw{ECz{tbXLBDAR$NRQ+{SH7QanN2hO{Ls_Bw9Ux1}fw zLbo-yr6~#-w|BNt6h*}CeQgXyvE%l^wsb{F==QO;OhqZ_9&cAD9myHTI~1x=a_;di1IRrh|S3H z9tG~$tUunXP=_`*AMaNXjJrFJTNE1N?!My=MWy5J!Q%sps?gnI$A=VJ#y!)=hZQ>F zo`vHh3ccf=<>RA@>d-wK$Hx^ljFf~E6N*|QCH2IlqRx?$abikQA4pKiOr)i@$0?6E(lXlPl}AHqx$TL{ zHpYG2_9W#o;y$E3S$W)XpT0dsc_MUQb9QFutV&tCcQd$^H?x}92m0%*LdXzQ?Q-7*gX%8`*PxUJuj0ZYTS(HxVfxc4?<+F|l z22Twrp9?)Oc4|mDz<6-_)UeV;Jh*UbL^hyVLyYu<(-X=Ui1gIc zlgbwz=^3Y|lrM$Sb5GAGhZ!vH=~?B=1PeJmr+mf1(x0AJz8YdRpI%gsFfuw%`;@N{ z8GWZi%GVtkgQu62Z-g?&POm6O84pdLURAn@hZat+DaRZSEuUUjz8QLG<8+K_oROK( z5vzKO$V}~sQ@!oT%;<<$y%WmJ?MPHjFt&0#l2q>!Tak`r)q9Su`i>OU`=PDP9cijb z#YsZ{k@C@a@gu9{`+;FGqu#H<{3MuO$L>h*tuY8R4q7mE}NQE z--dQ>m|9hfjO>KYHr01Tc4}w4>U&3aMrViWhfsEIXP3&y*v0MaR{4otNN0~K;Mk?_ z>{SIryP7-uRUyXi&Q6OeOziIKbf|uG>>lhKQ2i9zJ=Qs-T4Fph-8rm^5RWW$j;NL$ zk1ThNs(ubVve7xNT4CfQoS9JlLgb{LnN~&&;ZRC)miD zIn^Hyw*Jh#>dz3n`OKnfjgi}V#;5v=$n855QvL179Xzw7`X`h-c4kGj&UkeC%&IC% zJi2gZO|{{8botD>>fg|#8)stB7-n8VS1fvsCNH%s4!zcym(dlEUKh^G?Mg&rnR~ch zN$B;OJxEtFdV_P1zAFX2F}$a_D-DffKGxYqL2uGL*4M>AZ+1R5*p-gn5`Ju~D-(@p zK0e)*g_1OnFLY(23C_otyK>N5!;f!t<)Mkp{Dial=xv(()U$=??aus+v!&=A;r!gQ z%p&f&Zj`DiLeBM|G-r|iTrWxw7d4;jM;XlG&T|%&sVVL|=RhBD77v~q zKpzYjkDVJr)0rjH=Y~<1rexvV2%6z6Sw1(4J`^t5I5&=FGD{QAPoP^hrK#s9(QVGs zjPq0I_Hb$L`582e$>E-#MR#a8$oV<+VJAm_ejeQ!<}{yQM6;RP&htKWmxkMSK7{Ue zatF^Zp^t>QW9L`U9A??{`BjvyDO)(dhUPlUmd~%FkA}-O&c|SR%<_cpSZt4`JheLx zd(2s$(H)OH9xl)APQ>z=72NJ5>}8 zffYM>gWc&^Ntid*or#q)`P1E57)Qfj=+4HtPX2Ot4ptWCZ*=EjE&|=cCaO^@21~Em`7YrDz5iMM3#1N-w`9c#W3yU@`v|@55m~gQTQ)s}{i|v@w z31(dEz*J!{_hJ`@GR54B-5913BNuxx+$q*y?8Ve!ar4D~j9^MSFIq5-M$&iDfmJ#s zgBJ&|s<342;t-}~N~bRlV>*p=;o=CUcS@Hpj$+ke>BhxztcD3CT$;dYHBjoMNvzHZ zWn7xV>cdd(r5Vh?gt?byu?7u{T$;o7Ibr>!d2D|eZoagLH8PRTOFry?2I;#L!VWr- z!Ancnp)fLbX$5Oy%BC-^Vn&T@;nEt`?368ETE|+#vW-hIcq>z$&=ZRv*2q(P;_xF* zc}7n>el#r4?McMjm&e7Bn5yZXEZn3~E%apLolezqPY!-2tlH?w!@HPh!sUGYtOiZJT!^1@ zq8XP<@$+Fc_i{Pj&BVBu0sMjnLoS2(MJJ}e4B?l;So38$-owN@FQfQn4c>Q|z^^#* z!OL3wY8W58T!Z&A)zg;^xLKoKxZH^MIn~RToA4*Y>W#~-ct4X!xYC9{r6E$UwBt`Z ziHs{9_%mT5_evLTVQRQnx^b&UgIwvsZBC8;N-u5?Ynre0;|^wJ=M@X?)KvCeap2E7 zD+jL(;Ln9C$F2bO^D@s~9^;BTFr8x4wK1D+pkAANt)V{V&i|Ms zn^QKY(Be`y-LQE_Y}^f-|3}|+?f)2e9uFkmx%S;nw78uAFXiNav;TEsHm4*rtLxMm ziWgqMs`n0VX*%@2Y7|I5X7EQ}iC>2?#0^6xnW};abs#M`ScWv9Q fz_xDRz6;-$jicM~owy2T#uQu=6H~s2xAp%3?8Iq{ literal 0 HcmV?d00001 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 922c20fa71f73..4e00397f908f8 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 @@ -304,7 +304,7 @@ class ParquetFileFormat override def call(): ParquetFileSplitter = createParquetFileSplits(root, hadoopConf, schema) }) - root -> splits.buildSplitter(filters) + root -> splits.buildSplitter(filters, sparkSession.sessionState.conf) }.toMap val compositeSplitter: (FileStatus => Seq[FileSplit]) = { stat => val filePath = stat.getPath diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileSplitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileSplitter.scala index b17814e952c17..48e68fb169dac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileSplitter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileSplitter.scala @@ -25,19 +25,21 @@ import scala.concurrent.{ExecutionContext, Future} import com.google.common.cache.{Cache, CacheBuilder} import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.lib.input.FileSplit -import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate} +import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.filter2.statisticslevel.StatisticsFilter import org.apache.parquet.hadoop.metadata.BlockMetaData import org.roaringbitmap.RoaringBitmap import org.apache.spark.internal.Logging +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.ThreadUtils + abstract class ParquetFileSplitter { - def buildSplitter(filters: Seq[Filter]): (FileStatus => Seq[FileSplit]) + def buildSplitter(filters: Seq[Filter], sQLConf: SQLConf): (FileStatus => Seq[FileSplit]) def singleFileSplit(stat: FileStatus): Seq[FileSplit] = { Seq(new FileSplit(stat.getPath, 0, stat.getLen, Array.empty)) @@ -45,9 +47,9 @@ abstract class ParquetFileSplitter { } object ParquetDefaultFileSplitter extends ParquetFileSplitter { - override def buildSplitter(filters: Seq[Filter]): (FileStatus => Seq[FileSplit]) = { - stat => singleFileSplit(stat) - } + override def buildSplitter( + filters: Seq[Filter], + sQLConf: SQLConf): (FileStatus => Seq[FileSplit]) = singleFileSplit } class ParquetMetadataFileSplitter( @@ -65,7 +67,9 @@ class ParquetMetadataFileSplitter( .concurrencyLevel(1) .build() - override def buildSplitter(filters: Seq[Filter]): (FileStatus => Seq[FileSplit]) = { + override def buildSplitter( + filters: Seq[Filter], + sQLConf: SQLConf): (FileStatus => Seq[FileSplit]) = { val (applied, unapplied, filteredBlocks) = this.synchronized { val (applied, unapplied) = filters.partition(filterSets.getIfPresent(_) != null) val filteredBlocks = filterSets.getAllPresent(applied.asJava).values().asScala @@ -78,7 +82,8 @@ class ParquetMetadataFileSplitter( (applied, unapplied, filteredBlocks) } - val eligible = applyParquetFilter(unapplied, filteredBlocks).map { bmd => + val eligible = applyParquetFilter(unapplied, filteredBlocks, + sQLConf.isParquetINT96AsTimestamp).map { bmd => val blockPath = new Path(root, bmd.getPath) new FileSplit(blockPath, bmd.getStartingPos, bmd.getCompressedSize, Array.empty) } @@ -97,14 +102,15 @@ class ParquetMetadataFileSplitter( private def applyParquetFilter( filters: Seq[Filter], - blocks: Seq[BlockMetaData]): Seq[BlockMetaData] = { + blocks: Seq[BlockMetaData], + int96AsTimestamp: Boolean): Seq[BlockMetaData] = { val predicates = filters.flatMap { - ParquetFilters.createFilter(schema, _) + ParquetFilters.createFilter(schema, _, int96AsTimestamp) } if (predicates.nonEmpty) { // Asynchronously build bitmaps Future { - buildFilterBitMaps(filters) + buildFilterBitMaps(filters, int96AsTimestamp) }(ParquetMetadataFileSplitter.executionContext) val predicate = predicates.reduce(FilterApi.and) @@ -114,21 +120,21 @@ class ParquetMetadataFileSplitter( } } - private def buildFilterBitMaps(filters: Seq[Filter]): Unit = { + private def buildFilterBitMaps(filters: Seq[Filter], int96AsTimestamp: Boolean): Unit = { this.synchronized { // Only build bitmaps for filters that don't exist. val sets = filters .filter(filterSets.getIfPresent(_) == null) .flatMap { filter => val bitmap = new RoaringBitmap - ParquetFilters.createFilter(schema, filter) + ParquetFilters.createFilter(schema, filter, int96AsTimestamp) .map((filter, _, bitmap)) } var i = 0 val blockLen = blocks.size while (i < blockLen) { val bmd = blocks(i) - sets.foreach { case (filter, parquetFilter, bitmap) => + sets.foreach { case (_, parquetFilter, bitmap) => if (!StatisticsFilter.canDrop(parquetFilter, bmd.getColumns)) { bitmap.add(i) } 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 5562176c0feed..e6633eae0bef7 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 @@ -17,9 +17,10 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.filter2.predicate._ +import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.io.api.Binary + import org.apache.spark.sql.sources import org.apache.spark.sql.types._