diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 8bb0ce4b1b..4cc9bb30ca 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -931,7 +931,7 @@ object SplitSnappyClusterDUnitTest .set("snappydata.connection", connectionURL) .set("snapptdata.sql.planCaching", random.nextBoolean().toString) .set(Property.TestDisableCodeGenFlag.name, "false") - logInfo("Spark conf:" + conf.getAll.toString) + logInfo("Spark conf: " + conf.getAll.mkString(", ")) val sc = SparkContext.getOrCreate(conf) // sc.setLogLevel("DEBUG") diff --git a/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala b/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala index 40f1b9d9a8..da6300b652 100644 --- a/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala +++ b/cluster/src/dunit/scala/org/apache/spark/sql/ColumnBatchAndExternalTableDUnitTest.scala @@ -18,21 +18,22 @@ package org.apache.spark.sql import com.pivotal.gemfirexd.internal.engine.Misc -import io.snappydata.Property import io.snappydata.cluster.ClusterManagerTestBase import io.snappydata.test.dunit.{AvailablePortHelper, SerializableCallable} import io.snappydata.util.TestUtils +import io.snappydata.{Property, SnappyFunSuite} import org.scalatest.Assertions import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.sql.execution.ui.SQLExecutionUIData case class TestRecord(col1: Int, col2: Int, col3: Int) class ColumnBatchAndExternalTableDUnitTest(s: String) extends ClusterManagerTestBase(s) with Assertions with Logging with SparkSupport { - private def activeExecutionIds(session: SparkSession): Set[Long] = { + private def sqlExecutionIds(session: SparkSession): Set[Long] = { session.sharedState.statusStore.executionsList().map(_.executionId).toSet } @@ -53,64 +54,61 @@ class ColumnBatchAndExternalTableDUnitTest(s: String) extends ClusterManagerTest import session.implicits._ - val ds = session.createDataset(sc.range(1, 101).map(i => + val ds = session.createDataset(sc.range(1, 1001).map(i => AirlineData(2015, 2, 15, 1002, i.toInt, "AA" + i))) ds.write.insertInto("airline") // ***Check for the case when all the column batches are scanned **** - var previousExecutionIds = activeExecutionIds(session) + var previousExecutionIds = sqlExecutionIds(session) val df_allColumnBatchesScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + - "from AIRLINE where ArrDelay < 101 " + + "from AIRLINE where ArrDelay < 1001 " + "group by UniqueCarrier order by arrivalDelay") - df_allColumnBatchesScan.count() + df_allColumnBatchesScan.collect() - var executionIds = activeExecutionIds(session).diff(previousExecutionIds) + var executionIds = sqlExecutionIds(session).diff(previousExecutionIds) var executionId = executionIds.head - val (scanned1, skipped1) = - findColumnBatchStats(df_allColumnBatchesScan, session, executionId) + val (scanned1, skipped1) = findColumnBatchStats(session, executionId) assert(skipped1 == 0, "All Column batches should have been scanned") assert(scanned1 > 0, "All Column batches should have been scanned") // ***Check for the case when all the column batches are skipped**** - previousExecutionIds = activeExecutionIds(session) + previousExecutionIds = sqlExecutionIds(session) val df_noColumnBatchesScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + - "from AIRLINE where ArrDelay > 101 " + + "from AIRLINE where ArrDelay > 1001 " + "group by UniqueCarrier order by arrivalDelay") - df_noColumnBatchesScan.count() + df_noColumnBatchesScan.collect() - executionIds = activeExecutionIds(session).diff(previousExecutionIds) + executionIds = sqlExecutionIds(session).diff(previousExecutionIds) executionId = executionIds.head - val (scanned2, skipped2) = - findColumnBatchStats(df_allColumnBatchesScan, session, executionId) + val (scanned2, skipped2) = findColumnBatchStats(session, executionId) assert(scanned2 == skipped2, "No Column batches should have been scanned") assert(skipped2 > 0, "No Column batches should have been scanned") // ***Check for the case when some of the column batches are scanned **** - previousExecutionIds = activeExecutionIds(session) + previousExecutionIds = sqlExecutionIds(session) val df_someColumnBatchesScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + "from AIRLINE where ArrDelay < 20 " + "group by UniqueCarrier order by arrivalDelay") - df_someColumnBatchesScan.count() + df_someColumnBatchesScan.collect() - executionIds = activeExecutionIds(session).diff(previousExecutionIds) + executionIds = sqlExecutionIds(session).diff(previousExecutionIds) executionId = executionIds.head - val (scanned3, skipped3) = - findColumnBatchStats(df_allColumnBatchesScan, session, executionId) + val (scanned3, skipped3) = findColumnBatchStats(session, executionId) assert(skipped3 > 0, "Some Column batches should have been skipped") assert(scanned3 != skipped3, "Some Column batches should have been skipped - comparison") @@ -118,79 +116,86 @@ class ColumnBatchAndExternalTableDUnitTest(s: String) extends ClusterManagerTest // check for StartsWith predicate with MAX/MIN handling // first all batches chosen - previousExecutionIds = activeExecutionIds(session) + previousExecutionIds = sqlExecutionIds(session) val df_allColumnBatchesLikeScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + "from AIRLINE where UniqueCarrier like 'AA%' " + "group by UniqueCarrier order by arrivalDelay") - var count = df_allColumnBatchesLikeScan.count() - assert(count == 100, s"Unexpected count = $count, expected 100") + var count = df_allColumnBatchesLikeScan.collect().length + assert(count == 1000, s"Unexpected count = $count, expected 1000") - executionIds = activeExecutionIds(session).diff(previousExecutionIds) + executionIds = sqlExecutionIds(session).diff(previousExecutionIds) executionId = executionIds.head - val (scanned4, skipped4) = - findColumnBatchStats(df_allColumnBatchesLikeScan, session, executionId) + val (scanned4, skipped4) = findColumnBatchStats(session, executionId) assert(skipped4 == 0, "No Column batches should have been skipped") assert(scanned4 > 0, "All Column batches should have been scanned") // next some batches skipped - previousExecutionIds = activeExecutionIds(session) + previousExecutionIds = sqlExecutionIds(session) val df_someColumnBatchesLikeScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + "from AIRLINE where UniqueCarrier like 'AA1%' " + "group by UniqueCarrier order by arrivalDelay") - count = df_someColumnBatchesLikeScan.count() - assert(count == 12, s"Unexpected count = $count, expected 12") + count = df_someColumnBatchesLikeScan.collect().length + assert(count == 112, s"Unexpected count = $count, expected 112") - executionIds = activeExecutionIds(session).diff(previousExecutionIds) + executionIds = sqlExecutionIds(session).diff(previousExecutionIds) executionId = executionIds.head - val (scanned5, skipped5) = - findColumnBatchStats(df_someColumnBatchesLikeScan, session, executionId) + val (scanned5, skipped5) = findColumnBatchStats(session, executionId) assert(skipped5 > 0, "Some Column batches should have been skipped") assert(scanned5 != skipped5, "Some Column batches should have been skipped - comparison") // last all batches skipped - previousExecutionIds = activeExecutionIds(session) + previousExecutionIds = sqlExecutionIds(session) val df_noColumnBatchesLikeScan = session.sql( "select AVG(ArrDelay) arrivalDelay, UniqueCarrier carrier " + "from AIRLINE where UniqueCarrier like 'AA0%' " + "group by UniqueCarrier order by arrivalDelay") - count = df_noColumnBatchesLikeScan.count() + count = df_noColumnBatchesLikeScan.collect().length assert(count == 0, s"Unexpected count = $count, expected 0") - executionIds = activeExecutionIds(session).diff(previousExecutionIds) + executionIds = sqlExecutionIds(session).diff(previousExecutionIds) executionId = executionIds.head - val (scanned6, skipped6) = - findColumnBatchStats(df_noColumnBatchesLikeScan, session, executionId) + val (scanned6, skipped6) = findColumnBatchStats(session, executionId) assert(scanned6 == skipped6, "No Column batches should have been returned") assert(skipped6 > 0, "No Column batches should have been returned") } - private def findColumnBatchStats(df: DataFrame, - session: SnappySession, executionId: Long): (Long, Long) = { + private def getAccumulatorValue(execData: SQLExecutionUIData, name: String): Long = { + execData.metrics.find(_.name == name) match { + case Some(id) => execData.metricValues.get(id.accumulatorId) match { + case Some(v) => v.toLong + case _ => 0L + } + case _ => 0L + } + } - val execData = session.sharedState.statusStore.executionsList().find( - _.executionId == executionId).get - val seenId = execData.metrics.find(_.name == "column batches seen").get - val skippedId = execData.metrics.find(_.name == "column batches skipped by the predicate").get + private def findColumnBatchStats(session: SnappySession, executionId: Long): (Long, Long) = { + var execData: SQLExecutionUIData = null + SnappyFunSuite.waitForCriterion({ + execData = session.sharedState.statusStore.executionsList().find( + _.executionId == executionId).get + execData.metricValues ne null + }, s"waiting for metricValues of executionId = $executionId", 10000, 10) - (execData.metricValues.filter(_._1 == seenId).head._2.toInt, - execData.metricValues.filter(_._1 == skippedId).head._2.toInt) + (getAccumulatorValue(execData, "column batches seen"), + getAccumulatorValue(execData, "column batches skipped by the predicate")) } def testCreateColumnTablesFromOtherTables(): Unit = { diff --git a/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Column.scala b/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Column.scala index cb75bd1c26..04b4d206b1 100644 --- a/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Column.scala +++ b/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Column.scala @@ -1,3 +1,20 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ + package io.snappydata.filodb import scala.concurrent.duration.Duration diff --git a/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Row.scala b/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Row.scala index e2810bb283..d5a5c39c30 100644 --- a/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Row.scala +++ b/cluster/src/test/scala/io/snappydata/filodb/FiloDBApp_Row.scala @@ -1,3 +1,20 @@ +/* + * Copyright (c) 2017-2019 TIBCO Software Inc. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You + * may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + * implied. See the License for the specific language governing + * permissions and limitations under the License. See accompanying + * LICENSE file. + */ + package io.snappydata.filodb import java.sql.{DriverManager, PreparedStatement} diff --git a/core/src/main/scala/org/apache/spark/sql/internal/session.scala b/core/src/main/scala/org/apache/spark/sql/internal/session.scala index c034353461..991c0a9527 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/session.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/session.scala @@ -288,17 +288,23 @@ class SnappyConf(@transient val session: SnappySession) private[sql] def setDynamicCpusPerTask(): Unit = synchronized { if (dynamicCpusPerTask != -1) { + val numExecutors = SnappyContext.numExecutors + val totalUsableHeap = SnappyContext.foldLeftBlockIds(0L)(_ + _.usableHeapBytes) + + // skip for smart connector where there is no information of physical cores or heap + if (numExecutors == 0 || totalUsableHeap <= 0) return + val sparkCores = session.sparkContext.defaultParallelism.toDouble // calculate minimum required heap assuming a block size of 128M val minRequiredHeap = 128.0 * 1024.0 * 1024.0 * sparkCores * 1.2 - val totalUsableHeap = SnappyContext.foldLeftBlockIds(0L)(_ + _.usableHeapBytes) + // select bigger among (required heap / available) and (logical cores / physical) val cpusPerTask0 = math.max(minRequiredHeap / totalUsableHeap, sparkCores / SnappyContext.totalPhysicalCoreCount.get()) // keep a reasonable upper-limit so tasks can at least be scheduled: // used below is average logical cores / 2 val cpusPerTask = math.max(1, math.ceil(math.min(sparkCores / - (2 * SnappyContext.numExecutors), cpusPerTask0)).toInt) + (2 * numExecutors), cpusPerTask0)).toInt) setConfString(Constant.CPUS_PER_TASK_PROP, cpusPerTask.toString) dynamicCpusPerTask = cpusPerTask logDebug(s"Set dynamic ${Constant.CPUS_PER_TASK_PROP} to $cpusPerTask") diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/._common_metadata.crc b/tests/common/src/main/resources/2015-trimmed.parquet/._common_metadata.crc deleted file mode 100644 index 87f2da649d..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/._common_metadata.crc and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/._metadata.crc b/tests/common/src/main/resources/2015-trimmed.parquet/._metadata.crc deleted file mode 100644 index 36d573cffd..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/._metadata.crc and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc new file mode 100644 index 0000000000..eb1ee38a48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc new file mode 100644 index 0000000000..e3695b7226 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc new file mode 100644 index 0000000000..7ae4bd01c7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc new file mode 100644 index 0000000000..c189ef00a7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc new file mode 100644 index 0000000000..40bcfd049c Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc new file mode 100644 index 0000000000..6d5d341302 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc new file mode 100644 index 0000000000..b9660369f7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc new file mode 100644 index 0000000000..247f07f515 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc new file mode 100644 index 0000000000..1f56aa2418 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc new file mode 100644 index 0000000000..fa0fef5edc Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc new file mode 100644 index 0000000000..c4e4f793c8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc new file mode 100644 index 0000000000..3e43c094dd Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc new file mode 100644 index 0000000000..5dfa2c6b31 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc new file mode 100644 index 0000000000..67a47ef9df Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc new file mode 100644 index 0000000000..31dbee6681 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc new file mode 100644 index 0000000000..0e368c58fc Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc new file mode 100644 index 0000000000..67b8462871 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc new file mode 100644 index 0000000000..ea39d15a0f Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc new file mode 100644 index 0000000000..304a51bbff Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc new file mode 100644 index 0000000000..51b840db5e Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc new file mode 100644 index 0000000000..36a0d79b60 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc new file mode 100644 index 0000000000..df355abbb2 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc new file mode 100644 index 0000000000..791435df6a Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc new file mode 100644 index 0000000000..30b65a5f01 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc new file mode 100644 index 0000000000..4304a7916a Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc new file mode 100644 index 0000000000..c4b7187380 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc new file mode 100644 index 0000000000..f2fd1c4ed8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc new file mode 100644 index 0000000000..5592ab71ee Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc new file mode 100644 index 0000000000..1e197331c4 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc new file mode 100644 index 0000000000..166f7e10cd Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc new file mode 100644 index 0000000000..d7aa397c61 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc new file mode 100644 index 0000000000..a7faa52d84 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc new file mode 100644 index 0000000000..3f88d7f287 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc new file mode 100644 index 0000000000..222ca894e8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc new file mode 100644 index 0000000000..da5d49146c Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc new file mode 100644 index 0000000000..cd52eb55ff Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc new file mode 100644 index 0000000000..91e058d95d Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc new file mode 100644 index 0000000000..b0d6da9ac5 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc new file mode 100644 index 0000000000..ad6676d387 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc new file mode 100644 index 0000000000..cd5ebe68fb Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc new file mode 100644 index 0000000000..ff0c148f53 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc new file mode 100644 index 0000000000..187ffff0c2 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc new file mode 100644 index 0000000000..f524dd53df Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc new file mode 100644 index 0000000000..48948a7aba Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc new file mode 100644 index 0000000000..d3bb1d3b90 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc new file mode 100644 index 0000000000..17dcb4c94d Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc new file mode 100644 index 0000000000..943bb8d5ee Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc new file mode 100644 index 0000000000..c828d268a2 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc new file mode 100644 index 0000000000..0d82460ed7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc new file mode 100644 index 0000000000..68b161646d Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc new file mode 100644 index 0000000000..28e1a49b1f Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc new file mode 100644 index 0000000000..916c96036f Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc new file mode 100644 index 0000000000..0c654ad06b Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc new file mode 100644 index 0000000000..03538ed439 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc new file mode 100644 index 0000000000..2f88cc49d6 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc new file mode 100644 index 0000000000..e17972007e Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc new file mode 100644 index 0000000000..81c4f54fcb Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc new file mode 100644 index 0000000000..2ab78c843b Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc new file mode 100644 index 0000000000..e9b64c2763 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc new file mode 100644 index 0000000000..992cc60a6b Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc new file mode 100644 index 0000000000..d483ff631e Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc new file mode 100644 index 0000000000..28af7cfb34 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc new file mode 100644 index 0000000000..d14983a99c Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/.part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet.crc differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc deleted file mode 100644 index 5564fe463b..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc b/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc deleted file mode 100644 index cb7b3a4650..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/.part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet.crc and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/_common_metadata b/tests/common/src/main/resources/2015-trimmed.parquet/_common_metadata deleted file mode 100644 index 9beded5f10..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/_common_metadata and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/_metadata b/tests/common/src/main/resources/2015-trimmed.parquet/_metadata deleted file mode 100644 index 4c12e0ac71..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/_metadata and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet new file mode 100644 index 0000000000..9d02e85436 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00000-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet new file mode 100644 index 0000000000..59989f13e8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet new file mode 100644 index 0000000000..9a3c26b5ef Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet new file mode 100644 index 0000000000..4745a00c4b Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet new file mode 100644 index 0000000000..cc480f5ac4 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet new file mode 100644 index 0000000000..6e01ae011c Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet new file mode 100644 index 0000000000..f2c536e28a Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet new file mode 100644 index 0000000000..de0324a3a3 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet new file mode 100644 index 0000000000..ea155d5721 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet new file mode 100644 index 0000000000..4dd214c918 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet new file mode 100644 index 0000000000..077d7f5287 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet new file mode 100644 index 0000000000..0c5ecd2b1f Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet new file mode 100644 index 0000000000..76b676686f Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet new file mode 100644 index 0000000000..ca9d533969 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet new file mode 100644 index 0000000000..cb5a61c3c2 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet new file mode 100644 index 0000000000..f63cfc32b5 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet new file mode 100644 index 0000000000..6cab14bc16 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet new file mode 100644 index 0000000000..153bbe7396 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet new file mode 100644 index 0000000000..b75156e7dc Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet new file mode 100644 index 0000000000..66f082ef51 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet new file mode 100644 index 0000000000..b0d8d99ce6 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet new file mode 100644 index 0000000000..051078a1b7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet new file mode 100644 index 0000000000..cadcc5528a Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet new file mode 100644 index 0000000000..cdc3c5a4fa Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet new file mode 100644 index 0000000000..9674613ff9 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet new file mode 100644 index 0000000000..bfd7f00bcd Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet new file mode 100644 index 0000000000..466e006d03 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet new file mode 100644 index 0000000000..b907fb0590 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet new file mode 100644 index 0000000000..8b05dc9ad2 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet new file mode 100644 index 0000000000..09da836024 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet new file mode 100644 index 0000000000..8a83851286 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet new file mode 100644 index 0000000000..bba7633d8e Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00005-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet new file mode 100644 index 0000000000..3c65721cb1 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-00658b64-c850-4cf1-a62b-66df910b5c0d-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet new file mode 100644 index 0000000000..697b63824a Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-045af329-7928-4b99-bd5f-7509909cd629-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet new file mode 100644 index 0000000000..dd1adba2f1 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-0e960084-6914-452d-878d-025d567a705b-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet new file mode 100644 index 0000000000..8874eb1fb0 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-1986d9f8-af54-4ac3-9f96-ef9916a20cab-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet new file mode 100644 index 0000000000..e1836d6d85 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-21c7c705-a7d0-453c-a56c-2dc7cbe2cb44-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet new file mode 100644 index 0000000000..f92c54f219 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2521c941-34de-4532-af09-054c39814f92-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet new file mode 100644 index 0000000000..509f917128 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-2ba5ab7e-be2e-44a6-9deb-12e7f97895d3-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet new file mode 100644 index 0000000000..56b803778d Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-30e9e719-9951-4b82-9e15-a75b47e5f696-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet new file mode 100644 index 0000000000..ffcbcf72ea Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3103fd69-e890-4ce6-b27e-833c6ea8de80-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet new file mode 100644 index 0000000000..64f6f561e8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-35edb6a0-7e18-44d4-a026-80fc1e82b653-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet new file mode 100644 index 0000000000..ec454fdd36 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3b537503-41df-4274-956e-23869a4e7662-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet new file mode 100644 index 0000000000..5778b8dac8 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-3bcaf272-fc53-4775-8dba-5261c27ad67e-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet new file mode 100644 index 0000000000..57e21037fa Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4cd108be-a0f1-452f-a40a-2dd403ac79b9-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet new file mode 100644 index 0000000000..87dcf876fd Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-4ff6f25a-a5ab-46e3-83d9-f3f746510f64-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet new file mode 100644 index 0000000000..b7c2c38a83 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-504f4563-b66b-43a1-a095-4dfe8ffbb896-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet new file mode 100644 index 0000000000..9bac613a76 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-5e3f868d-ce23-4588-bc2e-54ac07b01b5c-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet new file mode 100644 index 0000000000..ce71b966ab Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7135ff90-7b82-4f5a-bd23-72e5f76f6225-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet new file mode 100644 index 0000000000..4b7244f65b Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7441115a-7e40-40d6-98d2-e1869f64bef2-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet new file mode 100644 index 0000000000..f6ecc1b9ee Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-7bc4ec26-3c17-4f4b-82d5-85bf626078a7-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet new file mode 100644 index 0000000000..f2df7c9e48 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-91e45d5f-cca3-44c7-8808-07f201d8ae97-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet new file mode 100644 index 0000000000..4e12f62afa Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-93d91940-764c-47f0-8afb-11ef79e699a5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet new file mode 100644 index 0000000000..f3daa58c87 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-97a8da5b-ad06-4911-9661-36dbb07821ae-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet new file mode 100644 index 0000000000..1b85733889 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-9b4e3257-cf23-40b2-9760-8a8ddd82bcc8-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet new file mode 100644 index 0000000000..0cc52a6633 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-aafbfa5b-c7bd-4c26-9e81-263d90950ea1-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet new file mode 100644 index 0000000000..7eb642e996 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ab26f8b7-91b6-45da-b9fe-28b4389acfdd-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet new file mode 100644 index 0000000000..521e492936 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-b4f5c52f-b8f2-4747-9399-da92c299179f-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet new file mode 100644 index 0000000000..14ffd64230 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d31f531d-8107-42cb-952b-250bc66fb332-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet new file mode 100644 index 0000000000..2f0d9922eb Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-d7b92535-c598-47e5-a5f9-e0494472c448-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet new file mode 100644 index 0000000000..d78f4efdc7 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-dfbe507d-5640-48be-9f2a-7504c4b3f1c5-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet new file mode 100644 index 0000000000..17c8f780f4 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-ed4153a2-dc32-4094-9d23-e3c6db36d2ec-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet new file mode 100644 index 0000000000..6870157c54 Binary files /dev/null and b/tests/common/src/main/resources/2015-trimmed.parquet/part-00006-f0b8feca-f48d-44d4-acbb-5a216f176ed4-c000.gz.parquet differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet deleted file mode 100644 index 041cd8f1f3..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00000-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet and /dev/null differ diff --git a/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet b/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet deleted file mode 100644 index 3f473e0c42..0000000000 Binary files a/tests/common/src/main/resources/2015-trimmed.parquet/part-r-00001-fa8c25b4-f2cf-4b87-ba3f-5181a4f50ee6.gz.parquet and /dev/null differ