From 2b69e5dd5c6986ad1ba4ac77c80aced41553c024 Mon Sep 17 00:00:00 2001 From: Yuan Date: Thu, 13 May 2021 20:49:21 +0800 Subject: [PATCH] [NSE-273]Support spark311 for branch 1.1.1 (#319) * [NSE-262] fix remainer loss in decimal divide (#263) * fix decimal divide int issue * correct cpp uts * use const reference Co-authored-by: Yuan Co-authored-by: Yuan * [NSE-261] ArrowDataSource: Add S3 Support (#270) Closes #261 * [NSE-196] clean up configs in unit tests (#271) * remove testing config * remove unused configs * [NSE-265] Reserve enough memory before UnsafeAppend in builder (#266) * change the UnsafeAppend to Append * fix buffer builder in shuffle shuffle builder use UnsafeAppend API for better performance. it tries to reserve enough space based on results of last recordbatch, this maybe not buggy if there's a dense recordbatch after a sparse one. this patch adds below fixes: - adds Reset() after Finish() in builder - reserve length for offset_builder in binary builder A further clean up on the reservation logic should be needed. Signed-off-by: Yuan Zhou Co-authored-by: Yuan Zhou * [NSE-274] Comment to trigger tpc-h RAM test (#275) Closes #274 * bump cmake to 3.16 (#281) Signed-off-by: Yuan Zhou * [NSE-276] Add option to switch Hadoop version (#277) Closes #276 * [NSE-119] clean up on comments (#288) Signed-off-by: Yuan Zhou * [NSE-206]Update installation guide and configuration guide. (#289) * [NSE-206]Update installation guide and configuration guide. * Fix numaBinding setting issue. & Update description for protobuf * [NSE-206]Fix Prerequisite and Arrow Installation Steps. (#290) * [NSE-245]Adding columnar RDD cache support (#246) * Adding columnar RDD cache support Signed-off-by: Chendi Xue * Directly save reference, only convert to Array[Byte] when calling by BlockManager Signed-off-by: Chendi Xue * Add DeAllocator to construction to make sure this instance will be released once it be deleted by JVM Signed-off-by: Chendi Xue * Delete cache by adding a release in InMemoryRelation Since unpersist only delete RDD object, seems our deAllocator wasn't being called along Now we added a release function in InMemoryRelation clearCache() func, may need to think a new way for 3.1.0 Signed-off-by: Chendi Xue * [NSE-207] fix issues found from aggregate unit tests (#233) * fix incorrect input in Expand * fix empty input for aggregate * fix only result expressions * fix empty aggregate expressions * fix res attr not found issue * refine * fix count distinct with null * fix groupby of NaN, -0.0 and 0.0 * fix count on mutiple cols with null in WSCG * format code * support normalize NaN and 0.0 * revert and update * support normalize function in WSCG * [NSE-206]Update documents and License for 1.1.0 (#292) * [NSE-206]Update documents and remove duplicate parts * Modify documents by comments * [NSE-293] fix unsafemap with key = '0' (#294) Signed-off-by: Yuan Zhou * [NSE-257] fix multiple slf4j bindings (#291) * [NSE-297] Disable incremental compiler in GHA CI (#298) Closes #297 * [NSE-285] ColumnarWindow: Support Date input in MAX/MIN (#286) Closes #285 * [NSE-304] Upgrade to Arrow 4.0.0: Change basic GHA TPC-H test target OAP Arrow branch (#306) * [NSE-302] remove exception (#303) * [NSE-273] support spark311 (#272) * support spark 3.0.2 Signed-off-by: Yuan Zhou * update to use spark 302 in unit tests Signed-off-by: Yuan Zhou * support spark 311 Signed-off-by: Yuan Zhou * fix Signed-off-by: Yuan Zhou * fix missing dep Signed-off-by: Yuan Zhou * fix broadcastexchange metrics Signed-off-by: Yuan Zhou * fix arrow data source Signed-off-by: Yuan Zhou * fix sum with decimal Signed-off-by: Yuan Zhou * fix c++ code Signed-off-by: Yuan Zhou * adding partial sum decimal sum Signed-off-by: Yuan Zhou * fix hashagg in wscg Signed-off-by: Yuan Zhou * fix partial sum with number type Signed-off-by: Yuan Zhou * fix AQE shuffle copy Signed-off-by: Yuan Zhou * fix shuffle redudant reat Signed-off-by: Yuan Zhou * fix rebase Signed-off-by: Yuan Zhou * fix format Signed-off-by: Yuan Zhou * avoid unecessary fallbacks Signed-off-by: Yuan Zhou * on-demand scala unit tests Signed-off-by: Yuan Zhou * clean up Signed-off-by: Yuan Zhou * [NSE-311] Build reports errors (#312) Closes #311 * [NSE-257] fix the dependency issue on v2 Co-authored-by: Rui Mo Co-authored-by: Hongze Zhang Co-authored-by: JiaKe Co-authored-by: Wei-Ting Chen Co-authored-by: Chendi.Xue Co-authored-by: Hong --- .github/workflows/report_ram_log.yml | 65 - .github/workflows/tpch.yml | 39 +- .github/workflows/unittests.yml | 9 +- .../CHANGELOG.md => CHANGELOG.md | 285 +- LICENSE | 1957 +++ README.md | 35 +- TPP.txt | 10639 ++++++++++++++++ arrow-data-source/LICENSE.txt | 201 - arrow-data-source/README.md | 18 +- .../vectorized/ArrowWritableColumnVector.java | 16 + .../docs/ApacheArrowInstallation.md | 70 - arrow-data-source/docs/Configuration.md | 29 - arrow-data-source/docs/Installation.md | 27 - arrow-data-source/docs/InstallationNotes.md | 47 - arrow-data-source/docs/OAP-Developer-Guide.md | 109 - .../docs/OAP-Installation-Guide.md | 69 - arrow-data-source/docs/Prerequisite.md | 151 - arrow-data-source/docs/SparkInstallation.md | 44 - arrow-data-source/docs/User-Guide.md | 118 - arrow-data-source/docs/image/columnar.png | Bin 43622 -> 0 bytes arrow-data-source/docs/image/core_arch.jpg | Bin 71904 -> 0 bytes arrow-data-source/docs/image/dataset.png | Bin 12562 -> 0 bytes ...ecision_support_bench1_result_by_query.png | Bin 32336 -> 0 bytes ...ecision_support_bench1_result_in_total.png | Bin 10412 -> 0 bytes ...ecision_support_bench2_result_by_query.png | Bin 25498 -> 0 bytes ...ecision_support_bench2_result_in_total.png | Bin 11741 -> 0 bytes arrow-data-source/docs/image/kernel.png | Bin 28613 -> 0 bytes .../docs/image/nativesql_arch.png | Bin 65151 -> 0 bytes arrow-data-source/docs/image/performance.png | Bin 81268 -> 0 bytes arrow-data-source/docs/image/shuffle.png | Bin 66318 -> 0 bytes arrow-data-source/docs/index.md | 118 - arrow-data-source/mkdocs.yml | 23 - .../parquet/ParquetFileFormat.scala | 5 +- arrow-data-source/pom.xml | 107 +- .../v2/arrow/ArrowDataSourceV2.scala | 2 +- .../datasources/v2/arrow/ArrowUtils.scala | 5 + .../arrow/ArrowDataSourceTest.scala | 15 +- docs/ApacheArrowInstallation.md | 28 +- docs/Configuration.md | 42 +- docs/Installation.md | 23 +- docs/OAP-Developer-Guide.md | 113 +- docs/OAP-Installation-Guide.md | 34 +- docs/Prerequisite.md | 54 +- docs/User-Guide.md | 30 +- .../image/arrowdatasource_validation.png | Bin docs/index.md | 32 +- mkdocs.yml | 2 +- native-sql-engine/core/pom.xml | 47 +- .../VectorizedParquetArrowReader.java | 2 +- .../com/intel/oap/ColumnarGuardRule.scala | 18 +- .../scala/com/intel/oap/ColumnarPlugin.scala | 73 +- .../com/intel/oap/ColumnarPluginConfig.scala | 8 +- .../ColumnarBroadcastHashJoinExec.scala | 16 +- .../oap/execution/ColumnarExpandExec.scala | 7 +- .../execution/ColumnarHashAggregateExec.scala | 168 +- .../execution/ColumnarInMemoryRelation.scala | 242 + .../ColumnarInMemoryTableScanExec.scala | 135 + .../ColumnarShuffledHashJoinExec.scala | 19 +- .../execution/ColumnarSortMergeJoinExec.scala | 10 +- .../ColumnarWholeStageCodegenExec.scala | 6 +- .../oap/execution/ColumnarWindowExec.scala | 348 +- .../execution/DataToArrowColumnarExec.scala | 1 - .../ColumnarConditionProjector.scala | 108 +- .../ColumnarConditionedProbeJoin.scala | 2 +- .../expression/ColumnarHashAggregation.scala | 58 +- .../expression/ColumnarSortMergeJoin.scala | 2 +- .../expression/ColumnarUnaryOperator.scala | 29 +- .../intel/oap/expression/ConverterUtils.scala | 51 +- .../shuffle/sort/ColumnarShuffleManager.scala | 35 +- .../sql/columnar/CachedBatchSerializer.scala | 348 + .../ColumnarBroadcastExchangeExec.scala | 8 +- .../ColumnarCollapseCodegenStages.scala | 11 +- .../sql/execution/ColumnarExchange.scala | 11 +- .../ColumnarShuffleExchangeExec.scala | 6 +- .../execution/ShuffledColumnarBatchRDD.scala | 8 +- .../adaptive/AdaptiveSparkPlanExec.scala | 620 - .../ColumnarCustomShuffleReaderExec.scala | 5 +- .../execution/adaptive/QueryStageExec.scala | 247 - .../execution/columnar/InMemoryRelation.scala | 287 + .../core/src/test/scala-fixme/README.md | 1 + .../shuffle/ColumnarShuffleWriterSuite.scala | 0 .../spark/sql/AggregateHashMapSuite.scala | 15 +- ...xCountDistinctForIntervalsQuerySuite.scala | 5 +- .../sql/ApproximatePercentileQuerySuite.scala | 5 +- .../apache/spark/sql/BenchmarkQueryTest.scala | 0 .../apache/spark/sql/CachedTableSuite.scala | 13 +- .../spark/sql/ColumnExpressionSuite.scala | 47 +- .../apache/spark/sql/ComplexTypesSuite.scala | 7 +- .../spark/sql/ConfigBehaviorSuite.scala | 5 +- .../sql/CountMinSketchAggQuerySuite.scala | 5 +- .../apache/spark/sql/CsvFunctionsSuite.scala | 5 +- .../spark/sql/DataFrameAggregateSuite.scala | 36 +- .../spark/sql/DataFrameComplexTypeSuite.scala | 5 +- .../spark/sql/DataFrameFunctionsSuite.scala | 5 +- .../apache/spark/sql/DataFrameHintSuite.scala | 5 +- .../spark/sql/DataFrameImplicitsSuite.scala | 5 +- .../apache/spark/sql/DataFrameJoinSuite.scala | 7 +- .../spark/sql/DataFrameNaFunctionsSuite.scala | 7 +- .../spark/sql/DataFramePivotSuite.scala | 5 +- .../spark/sql/DataFrameRangeSuite.scala | 7 +- .../spark/sql/DataFrameSelfJoinSuite.scala | 5 +- .../sql/DataFrameSetOperationsSuite.scala | 5 +- .../apache/spark/sql/DataFrameStatSuite.scala | 5 +- .../org/apache/spark/sql/DataFrameSuite.scala | 9 +- .../sql/DataFrameTimeWindowingSuite.scala | 3 - .../spark/sql/DataFrameTungstenSuite.scala | 5 +- .../sql/DataFrameWindowFramesSuite.scala | 5 +- .../sql/DataFrameWindowFunctionsSuite.scala | 5 +- .../spark/sql/DataFrameWriterV2Suite.scala | 35 +- .../spark/sql/DatasetAggregatorSuite.scala | 5 +- .../apache/spark/sql/DatasetCacheSuite.scala | 7 +- .../spark/sql/DatasetOptimizationSuite.scala | 5 +- .../spark/sql/DatasetPrimitiveSuite.scala | 5 +- .../DatasetSerializerRegistratorSuite.scala | 5 +- .../org/apache/spark/sql/DatasetSuite.scala | 9 +- .../apache/spark/sql/DateFunctionsSuite.scala | 5 +- .../apache/spark/sql/DeprecatedAPISuite.scala | 7 +- .../DeprecatedDatasetAggregatorSuite.scala | 5 +- .../sql/DynamicPartitionPruningSuite.scala | 5 +- .../org/apache/spark/sql/ExplainSuite.scala | 14 +- .../spark/sql/ExpressionsSchemaSuite.scala | 5 +- .../spark/sql/ExtraStrategiesSuite.scala | 5 +- .../spark/sql/FileBasedDataSourceSuite.scala | 7 +- .../spark/sql/GeneratorFunctionSuite.scala | 5 +- .../spark/sql/IntegratedUDFTestUtils.scala | 0 .../org/apache/spark/sql/JoinHintSuite.scala | 7 +- .../org/apache/spark/sql/JoinSuite.scala | 5 +- .../apache/spark/sql/JsonFunctionsSuite.scala | 5 +- .../apache/spark/sql/LocalSparkSession.scala | 0 .../apache/spark/sql/MathFunctionsSuite.scala | 5 +- .../apache/spark/sql/MetadataCacheSuite.scala | 5 +- .../apache/spark/sql/MiscFunctionsSuite.scala | 5 +- .../spark/sql/ProcessingTimeSuite.scala | 0 .../org/apache/spark/sql/QueryTest.scala | 0 .../apache/spark/sql/RepartitionSuite.scala | 0 ...ullWithFalseInPredicateEndToEndSuite.scala | 5 +- .../org/apache/spark/sql/RowSuite.scala | 5 +- .../apache/spark/sql/RuntimeConfigSuite.scala | 0 .../apache/spark/sql/SQLContextSuite.scala | 0 .../org/apache/spark/sql/SQLQuerySuite.scala | 9 +- .../apache/spark/sql/SQLQueryTestSuite.scala | 0 .../org/apache/spark/sql/SSBQuerySuite.scala | 0 .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../apache/spark/sql/SerializationSuite.scala | 0 .../apache/spark/sql/SessionStateSuite.scala | 0 .../spark/sql/ShowCreateTableSuite.scala | 0 .../spark/sql/SparkSessionBuilderSuite.scala | 0 .../sql/SparkSessionExtensionSuite.scala | 0 .../spark/sql/StatisticsCollectionSuite.scala | 11 +- .../sql/StatisticsCollectionTestBase.scala | 0 .../spark/sql/StringFunctionsSuite.scala | 5 +- .../org/apache/spark/sql/SubquerySuite.scala | 9 +- .../apache/spark/sql/TPCDSQuerySuite.scala | 5 +- .../org/apache/spark/sql/TPCDSSchema.scala | 0 .../org/apache/spark/sql/TPCHQuerySuite.scala | 5 +- .../sql/TestQueryExecutionListener.scala | 0 .../sql/TypedImperativeAggregateSuite.scala | 5 +- .../org/apache/spark/sql/UDFSuite.scala | 7 +- .../spark/sql/UDTRegistrationSuite.scala | 0 .../org/apache/spark/sql/UnsafeRowSuite.scala | 0 .../spark/sql/UserDefinedTypeSuite.scala | 7 +- .../spark/sql/XPathFunctionsSuite.scala | 5 +- .../spark/sql/connector/AlterTableTests.scala | 5 +- ...SourceV2DataFrameSessionCatalogSuite.scala | 7 +- .../DataSourceV2DataFrameSuite.scala | 25 +- .../DataSourceV2SQLSessionCatalogSuite.scala | 7 +- .../sql/connector/DataSourceV2SQLSuite.scala | 35 +- .../sql/connector/DataSourceV2Suite.scala | 7 +- .../connector/DataSourceV2UtilsSuite.scala | 0 .../FileDataSourceV2FallBackSuite.scala | 7 +- .../spark/sql/connector/InsertIntoTests.scala | 51 +- .../connector/SimpleWritableDataSource.scala | 0 .../SupportsCatalogOptionsSuite.scala | 11 +- .../connector/TableCapabilityCheckSuite.scala | 5 +- .../connector/TestV2SessionCatalogBase.scala | 0 .../sql/connector/V1ReadFallbackSuite.scala | 5 +- .../sql/connector/V1WriteFallbackSuite.scala | 10 +- .../V2CommandsCaseSensitivitySuite.scala | 5 +- .../AggregatingAccumulatorSuite.scala | 5 +- .../execution/BroadcastExchangeSuite.scala | 5 +- .../execution/CoGroupedIteratorSuite.scala | 0 .../CoalesceShufflePartitionsSuite.scala | 0 .../DataSourceScanExecRedactionSuite.scala | 14 +- .../DeprecatedWholeStageCodegenSuite.scala | 0 .../spark/sql/execution/ExchangeSuite.scala | 5 +- ...xternalAppendOnlyUnsafeRowArraySuite.scala | 0 .../sql/execution/GlobalTempViewSuite.scala | 5 +- .../sql/execution/GroupedIteratorSuite.scala | 0 .../spark/sql/execution/HiveResultSuite.scala | 5 +- .../LogicalPlanTagInSparkPlanSuite.scala | 0 .../OptimizeMetadataOnlyQuerySuite.scala | 5 +- .../spark/sql/execution/PlannerSuite.scala | 5 +- .../sql/execution/QueryExecutionSuite.scala | 5 +- .../QueryPlanningTrackerEndToEndSuite.scala | 5 +- .../spark/sql/execution/ReferenceSort.scala | 0 .../sql/execution/SQLExecutionSuite.scala | 0 .../sql/execution/SQLJsonProtocolSuite.scala | 0 .../spark/sql/execution/SQLViewSuite.scala | 7 +- .../execution/SQLWindowFunctionSuite.scala | 5 +- .../spark/sql/execution/SameResultSuite.scala | 5 +- .../ShufflePartitionsUtilSuite.scala | 0 .../spark/sql/execution/SortSuite.scala | 6 +- .../spark/sql/execution/SparkPlanSuite.scala | 5 +- .../spark/sql/execution/SparkPlanTest.scala | 0 .../sql/execution/SparkPlannerSuite.scala | 5 +- .../sql/execution/SparkSqlParserSuite.scala | 0 .../TakeOrderedAndProjectSuite.scala | 5 +- .../UnsafeFixedWidthAggregationMapSuite.scala | 5 +- .../UnsafeKVExternalSorterSuite.scala | 5 +- .../execution/UnsafeRowSerializerSuite.scala | 0 .../WholeStageCodegenSparkSubmitSuite.scala | 0 .../execution/WholeStageCodegenSuite.scala | 7 +- .../adaptive/AdaptiveQueryExecSuite.scala | 5 +- .../adaptive/AdaptiveTestUtils.scala | 0 .../ColumnarAdaptiveQueryExecSuite.scala | 0 .../SortBasedAggregationStoreSuite.scala | 0 .../arrow/ArrowConvertersSuite.scala | 5 +- .../execution/arrow/ArrowWriterSuite.scala | 0 .../execution/columnar/ColumnStatsSuite.scala | 0 .../execution/columnar/ColumnTypeSuite.scala | 0 .../columnar/ColumnarTestUtils.scala | 0 .../columnar/InMemoryColumnarQuerySuite.scala | 7 +- .../NullableColumnAccessorSuite.scala | 0 .../columnar/NullableColumnBuilderSuite.scala | 0 .../columnar/PartitionBatchPruningSuite.scala | 5 +- .../compression/BooleanBitSetSuite.scala | 0 .../compression/DictionaryEncodingSuite.scala | 0 .../compression/IntegralDeltaSuite.scala | 0 .../PassThroughEncodingSuite.scala | 0 .../compression/RunLengthEncodingSuite.scala | 0 .../TestCompressibleColumnBuilder.scala | 0 .../execution/command/CommandUtilsSuite.scala | 0 .../execution/command/DDLParserSuite.scala | 5 +- .../sql/execution/command/DDLSuite.scala | 13 +- .../command/PlanResolutionSuite.scala | 0 .../BasicWriteTaskStatsTrackerSuite.scala | 0 .../datasources/BucketingUtilsSuite.scala | 0 .../datasources/DataSourceStrategySuite.scala | 5 +- .../datasources/FileBasedDataSourceTest.scala | 0 .../datasources/FileFormatWriterSuite.scala | 7 +- .../datasources/FileIndexSuite.scala | 5 +- .../datasources/FileSourceStrategySuite.scala | 7 +- .../HadoopFileLinesReaderSuite.scala | 5 +- .../datasources/ReadNestedSchemaTest.scala | 0 .../datasources/ReadSchemaSuite.scala | 63 +- .../datasources/ReadSchemaTest.scala | 0 .../RowDataSourceStrategySuite.scala | 5 +- .../SaveIntoDataSourceCommandSuite.scala | 5 +- .../datasources/SchemaPruningSuite.scala | 0 .../binaryfile/BinaryFileFormatSuite.scala | 5 +- .../execution/datasources/csv/CSVSuite.scala | 25 +- .../datasources/csv/TestCsvData.scala | 0 .../datasources/jdbc/JdbcUtilsSuite.scala | 0 .../json/JsonParsingOptionsSuite.scala | 5 +- .../datasources/json/JsonSuite.scala | 9 +- .../datasources/json/TestJsonData.scala | 0 .../datasources/noop/NoopStreamSuite.scala | 5 +- .../datasources/noop/NoopSuite.scala | 7 +- .../orc/OrcColumnarBatchReaderSuite.scala | 5 +- .../orc/OrcPartitionDiscoverySuite.scala | 14 +- .../datasources/orc/OrcQuerySuite.scala | 11 +- .../datasources/orc/OrcSourceSuite.scala | 7 +- .../execution/datasources/orc/OrcTest.scala | 0 .../datasources/orc/OrcV1FilterSuite.scala | 5 +- .../orc/OrcV1SchemaPruningSuite.scala | 7 +- .../orc/OrcV2SchemaPruningSuite.scala | 7 +- .../ParquetAvroCompatibilitySuite.scala | 7 +- .../parquet/ParquetCommitterSuite.scala | 0 .../parquet/ParquetCompatibilityTest.scala | 0 ...rquetCompressionCodecPrecedenceSuite.scala | 5 +- .../parquet/ParquetEncodingSuite.scala | 5 +- .../parquet/ParquetFileFormatSuite.scala | 5 +- .../parquet/ParquetFilterSuite.scala | 7 +- .../datasources/parquet/ParquetIOSuite.scala | 7 +- .../ParquetInteroperabilitySuite.scala | 7 +- .../ParquetPartitionDiscoverySuite.scala | 7 +- .../ParquetProtobufCompatibilitySuite.scala | 7 +- .../parquet/ParquetQuerySuite.scala | 7 +- .../parquet/ParquetSchemaPruningSuite.scala | 14 +- .../parquet/ParquetSchemaSuite.scala | 14 +- .../datasources/parquet/ParquetTest.scala | 0 .../ParquetThriftCompatibilitySuite.scala | 7 +- .../datasources/text/TextSuite.scala | 7 +- .../datasources/text/WholeTextFileSuite.scala | 14 +- .../datasources/v2/FileTableSuite.scala | 5 +- .../v2/V2SessionCatalogSuite.scala | 10 +- .../sql/execution/debug/DebuggingSuite.scala | 0 .../history/SQLEventFilterBuilderSuite.scala | 0 .../SQLLiveEntitiesEventFilterSuite.scala | 0 .../execution/joins/BroadcastJoinSuite.scala | 0 .../execution/joins/ExistenceJoinSuite.scala | 5 +- .../execution/joins/HashedRelationSuite.scala | 5 +- .../sql/execution/joins/InnerJoinSuite.scala | 5 +- .../sql/execution/joins/OuterJoinSuite.scala | 9 +- .../execution/metric/SQLMetricsSuite.scala | 5 +- .../metric/SQLMetricsTestUtils.scala | 0 .../python/BatchEvalPythonExecSuite.scala | 5 +- .../python/ExtractPythonUDFsSuite.scala | 5 +- .../python/PythonForeachWriterSuite.scala | 0 .../sql/execution/python/PythonUDFSuite.scala | 5 +- .../sql/execution/python/RowQueueSuite.scala | 0 .../CheckpointFileManagerSuite.scala | 5 +- .../CompactibleFileStreamLogSuite.scala | 5 +- .../streaming/FileStreamSinkLogSuite.scala | 5 +- .../streaming/HDFSMetadataLogSuite.scala | 5 +- .../execution/streaming/MemorySinkSuite.scala | 5 +- .../streaming/MicroBatchExecutionSuite.scala | 5 +- .../streaming/OffsetSeqLogSuite.scala | 5 +- .../ProcessingTimeExecutorSuite.scala | 0 .../streaming/StreamMetadataSuite.scala | 5 +- .../shuffle/ContinuousShuffleSuite.scala | 5 +- .../sources/ConsoleWriteSupportSuite.scala | 5 +- .../sources/ForeachBatchSinkSuite.scala | 5 +- .../sources/ForeachWriterSuite.scala | 5 +- .../sources/RateStreamProviderSuite.scala | 5 +- .../sources/TextSocketStreamSuite.scala | 5 +- ...latMapGroupsWithStateExecHelperSuite.scala | 0 .../streaming/state/MemoryStateStore.scala | 0 .../state/StateStoreCoordinatorSuite.scala | 0 .../streaming/state/StateStoreRDDSuite.scala | 0 .../streaming/state/StateStoreSuite.scala | 0 ...treamingAggregationStateManagerSuite.scala | 0 .../SymmetricHashJoinStateManagerSuite.scala | 0 .../execution/ui/AllExecutionsPageSuite.scala | 5 +- .../ui/SQLAppStatusListenerSuite.scala | 5 +- .../vectorized/ArrowColumnVectorSuite.scala | 0 .../vectorized/ColumnVectorSuite.scala | 0 .../vectorized/ColumnarBatchSuite.scala | 0 .../sql/expressions/ExpressionInfoSuite.scala | 5 +- .../expressions/ReduceAggregatorSuite.scala | 0 .../spark/sql/internal/CatalogSuite.scala | 5 +- .../DeprecatedCreateExternalTableSuite.scala | 5 +- .../internal/ExecutorSideSQLConfSuite.scala | 0 .../sql/internal/SQLConfEntrySuite.scala | 0 .../sql/internal/SQLConfGetterSuite.scala | 0 .../spark/sql/internal/SQLConfSuite.scala | 9 +- .../spark/sql/internal/SharedStateSuite.scala | 5 +- .../internal/VariableSubstitutionSuite.scala | 0 .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 5 +- .../spark/sql/jdbc/JDBCWriteSuite.scala | 7 +- .../spark/sql/sources/BucketedReadSuite.scala | 7 +- .../sql/sources/BucketedWriteSuite.scala | 5 +- .../sources/CreateTableAsSelectSuite.scala | 5 +- .../sql/sources/DDLSourceLoadSuite.scala | 5 +- .../sql/sources/DataSourceAnalysisSuite.scala | 0 .../spark/sql/sources/DataSourceTest.scala | 0 .../sources/ExternalCommandRunnerSuite.scala | 5 +- .../spark/sql/sources/FilteredScanSuite.scala | 5 +- .../spark/sql/sources/FiltersSuite.scala | 0 .../spark/sql/sources/InsertSuite.scala | 7 +- .../sql/sources/PartitionedWriteSuite.scala | 7 +- .../spark/sql/sources/PathOptionSuite.scala | 5 +- .../spark/sql/sources/PrunedScanSuite.scala | 5 +- .../sql/sources/ResolvedDataSourceSuite.scala | 5 +- .../spark/sql/sources/SaveLoadSuite.scala | 7 +- .../spark/sql/sources/TableScanSuite.scala | 7 +- .../sql/sources/fakeExternalSources.scala | 0 .../DeprecatedStreamingAggregationSuite.scala | 5 +- .../streaming/EventTimeWatermarkSuite.scala | 5 +- .../sql/streaming/FileStreamSinkSuite.scala | 5 +- .../sql/streaming/FileStreamSourceSuite.scala | 14 +- .../sql/streaming/FileStreamStressSuite.scala | 5 +- .../FlatMapGroupsWithStateSuite.scala | 5 +- .../streaming/MemorySourceStressSuite.scala | 5 +- .../spark/sql/streaming/OffsetSuite.scala | 0 .../sql/streaming/StateStoreMetricsTest.scala | 0 .../spark/sql/streaming/StreamSuite.scala | 7 +- .../spark/sql/streaming/StreamTest.scala | 0 .../streaming/StreamingAggregationSuite.scala | 5 +- .../StreamingDeduplicationSuite.scala | 5 +- .../sql/streaming/StreamingJoinSuite.scala | 10 +- .../StreamingQueryListenerSuite.scala | 5 +- .../StreamingQueryListenersConfSuite.scala | 5 +- .../StreamingQueryManagerSuite.scala | 5 +- ...StreamingQueryStatusAndProgressSuite.scala | 5 +- .../sql/streaming/StreamingQuerySuite.scala | 7 +- ...treamingSymmetricHashJoinHelperSuite.scala | 5 +- .../ContinuousAggregationSuite.scala | 5 +- ...ontinuousQueryStatusAndProgressSuite.scala | 0 .../ContinuousQueuedDataReaderSuite.scala | 5 +- .../continuous/ContinuousSuite.scala | 5 +- .../continuous/EpochCoordinatorSuite.scala | 0 .../sources/StreamingDataSourceV2Suite.scala | 5 +- .../test/DataStreamReaderWriterSuite.scala | 5 +- .../ui/StreamingQueryPageSuite.scala | 5 +- .../StreamingQueryStatusListenerSuite.scala | 5 +- .../sql/streaming/ui/UISeleniumSuite.scala | 0 .../spark/sql/streaming/ui/UIUtilsSuite.scala | 0 .../streaming/util/BlockOnStopSource.scala | 0 .../sql/streaming/util/BlockingSource.scala | 0 .../streaming/util/MockSourceProvider.scala | 0 .../streaming/util/StreamManualClock.scala | 0 .../sql/test/DataFrameReaderWriterSuite.scala | 7 +- .../spark/sql/test/GenericFlatSpecSuite.scala | 5 +- .../spark/sql/test/GenericFunSpecSuite.scala | 5 +- .../spark/sql/test/GenericWordSpecSuite.scala | 5 +- .../spark/sql/test/ProcessTestUtils.scala | 0 .../apache/spark/sql/test/SQLTestData.scala | 0 .../apache/spark/sql/test/SQLTestUtils.scala | 2 +- .../spark/sql/test/SharedSparkSession.scala | 0 .../spark/sql/test/TestSQLContext.scala | 0 .../sql/test/TestSparkSessionSuite.scala | 0 .../TravisDataFrameAggregateSuite.scala | 70 +- .../sql/travis/TravisDataFrameJoinSuite.scala | 5 +- .../sql/util/DataFrameCallbackSuite.scala | 5 +- .../util/ExecutionListenerManagerSuite.scala | 0 .../com/intel/oap/tpc/ds/TPCDSSuite.scala | 11 +- .../intel/oap/tpc/h/GitHubActionEntries.scala | 31 +- .../scala/com/intel/oap/tpc/h/TPCHSuite.scala | 1 - native-sql-engine/cpp/CMakeLists.txt | 2 +- native-sql-engine/cpp/src/CMakeLists.txt | 2 +- .../codegen/arrow_compute/ext/actions_impl.cc | 457 +- .../codegen/arrow_compute/ext/actions_impl.h | 7 +- .../ext/expression_codegen_visitor.cc | 21 + .../ext/hash_aggregate_kernel.cc | 69 +- .../codegen/arrow_compute/ext/kernels_ext.h | 4 + .../arrow_compute/ext/window_kernel.cc | 7 + .../cpp/src/codegen/common/hash_relation.h | 6 +- .../cpp/src/precompile/gandiva.h | 17 + native-sql-engine/cpp/src/shuffle/splitter.cc | 11 +- .../src/tests/arrow_compute_test_aggregate.cc | 78 + .../arrow_compute_test_aggregate_decimal.cc | 10 +- .../tests/arrow_compute_test_precompile.cc | 11 + .../cpp/src/tests/arrow_compute_test_wscg.cc | 74 + .../src/third_party/gandiva/decimal_ops.cc | 17 + .../cpp/src/third_party/gandiva/decimal_ops.h | 3 + .../third_party/sparsehash/sparse_hash_map.h | 82 +- pom.xml | 110 + scalastyle-config.xml | 6 - 429 files changed, 16338 insertions(+), 3837 deletions(-) delete mode 100644 .github/workflows/report_ram_log.yml rename arrow-data-source/CHANGELOG.md => CHANGELOG.md (54%) create mode 100644 LICENSE create mode 100644 TPP.txt delete mode 100644 arrow-data-source/LICENSE.txt delete mode 100644 arrow-data-source/docs/ApacheArrowInstallation.md delete mode 100644 arrow-data-source/docs/Configuration.md delete mode 100644 arrow-data-source/docs/Installation.md delete mode 100644 arrow-data-source/docs/InstallationNotes.md delete mode 100644 arrow-data-source/docs/OAP-Developer-Guide.md delete mode 100644 arrow-data-source/docs/OAP-Installation-Guide.md delete mode 100644 arrow-data-source/docs/Prerequisite.md delete mode 100644 arrow-data-source/docs/SparkInstallation.md delete mode 100644 arrow-data-source/docs/User-Guide.md delete mode 100644 arrow-data-source/docs/image/columnar.png delete mode 100644 arrow-data-source/docs/image/core_arch.jpg delete mode 100644 arrow-data-source/docs/image/dataset.png delete mode 100644 arrow-data-source/docs/image/decision_support_bench1_result_by_query.png delete mode 100644 arrow-data-source/docs/image/decision_support_bench1_result_in_total.png delete mode 100644 arrow-data-source/docs/image/decision_support_bench2_result_by_query.png delete mode 100644 arrow-data-source/docs/image/decision_support_bench2_result_in_total.png delete mode 100644 arrow-data-source/docs/image/kernel.png delete mode 100644 arrow-data-source/docs/image/nativesql_arch.png delete mode 100644 arrow-data-source/docs/image/performance.png delete mode 100644 arrow-data-source/docs/image/shuffle.png delete mode 100644 arrow-data-source/docs/index.md delete mode 100644 arrow-data-source/mkdocs.yml rename {arrow-data-source/docs => docs}/image/arrowdatasource_validation.png (100%) create mode 100644 native-sql-engine/core/src/main/scala/com/intel/oap/execution/ColumnarInMemoryRelation.scala create mode 100644 native-sql-engine/core/src/main/scala/com/intel/oap/execution/ColumnarInMemoryTableScanExec.scala create mode 100644 native-sql-engine/core/src/main/scala/org/apache/spark/sql/columnar/CachedBatchSerializer.scala delete mode 100644 native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala delete mode 100644 native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala create mode 100644 native-sql-engine/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala create mode 100644 native-sql-engine/core/src/test/scala-fixme/README.md rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/shuffle/ColumnarShuffleWriterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/AggregateHashMapSuite.scala (88%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ApproxCountDistinctForIntervalsQuerySuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/BenchmarkQueryTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/CachedTableSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ColumnExpressionSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ComplexTypesSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ConfigBehaviorSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/CountMinSketchAggQuerySuite.scala (90%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/CsvFunctionsSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameAggregateSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameComplexTypeSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameFunctionsSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameHintSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameImplicitsSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameJoinSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFramePivotSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameRangeSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameSelfJoinSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameSetOperationsSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameStatSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameTimeWindowingSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameTungstenSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameWindowFramesSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DataFrameWriterV2Suite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetAggregatorSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetCacheSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetOptimizationSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetPrimitiveSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DatasetSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DateFunctionsSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DeprecatedAPISuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DeprecatedDatasetAggregatorSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/DynamicPartitionPruningSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ExplainSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ExpressionsSchemaSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ExtraStrategiesSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/FileBasedDataSourceSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/GeneratorFunctionSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/IntegratedUDFTestUtils.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/JoinHintSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/JoinSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/JsonFunctionsSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/LocalSparkSession.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/MathFunctionsSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/MetadataCacheSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/MiscFunctionsSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ProcessingTimeSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/QueryTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/RepartitionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/RowSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/RuntimeConfigSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SQLContextSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SQLQuerySuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SQLQueryTestSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SSBQuerySuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ScalaReflectionRelationSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SerializationSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SessionStateSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/ShowCreateTableSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SparkSessionBuilderSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SparkSessionExtensionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/StatisticsCollectionSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/StatisticsCollectionTestBase.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/StringFunctionsSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/SubquerySuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/TPCDSQuerySuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/TPCDSSchema.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/TPCHQuerySuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/TestQueryExecutionListener.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/TypedImperativeAggregateSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/UDFSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/UDTRegistrationSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/UnsafeRowSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/UserDefinedTypeSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/XPathFunctionsSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/AlterTableTests.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2DataFrameSessionCatalogSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2DataFrameSuite.scala (90%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2SQLSessionCatalogSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2Suite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/DataSourceV2UtilsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/FileDataSourceV2FallBackSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/InsertIntoTests.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/SimpleWritableDataSource.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/SupportsCatalogOptionsSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/V1ReadFallbackSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/V1WriteFallbackSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/connector/V2CommandsCaseSensitivitySuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/AggregatingAccumulatorSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/CoGroupedIteratorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/DeprecatedWholeStageCodegenSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ExchangeSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArraySuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/GlobalTempViewSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/GroupedIteratorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/HiveResultSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/PlannerSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/QueryExecutionSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ReferenceSort.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SQLExecutionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SQLViewSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SameResultSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ShufflePartitionsUtilSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SortSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SparkPlanSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SparkPlanTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SparkPlannerSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/SparkSqlParserSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/adaptive/AdaptiveTestUtils.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/adaptive/ColumnarAdaptiveQueryExecSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/aggregate/SortBasedAggregationStoreSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/arrow/ArrowConvertersSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/arrow/ArrowWriterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/ColumnTypeSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/ColumnarTestUtils.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/NullableColumnAccessorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/NullableColumnBuilderSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/BooleanBitSetSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/DictionaryEncodingSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/RunLengthEncodingSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/columnar/compression/TestCompressibleColumnBuilder.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/command/CommandUtilsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/command/DDLParserSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/command/DDLSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/BucketingUtilsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/DataSourceStrategySuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/HadoopFileLinesReaderSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/ReadNestedSchemaTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala (83%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/RowDataSourceStrategySuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/csv/TestCsvData.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtilsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala (90%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcV1SchemaPruningSuite.scala (87%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetAvroCompatibilitySuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetCommitterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetCompatibilityTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetCompressionCodecPrecedenceSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetEncodingSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormatSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetProtobufCompatibilitySuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala (88%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/parquet/ParquetThriftCompatibilitySuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/text/TextSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/text/WholeTextFileSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/debug/DebuggingSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/history/SQLEventFilterBuilderSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/history/SQLLiveEntitiesEventFilterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/metric/SQLMetricsTestUtils.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/python/PythonUDFSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/python/RowQueueSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/CheckpointFileManagerSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/FileStreamSinkLogSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala (93%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/OffsetSeqLogSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/StreamMetadataSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/FlatMapGroupsWithStateExecHelperSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/StateStoreRDDSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/StreamingAggregationStateManagerSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManagerSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/expressions/ExpressionInfoSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/expressions/ReduceAggregatorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/CatalogSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/DeprecatedCreateExternalTableSuite.scala (94%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/ExecutorSideSQLConfSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/SQLConfEntrySuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/SQLConfGetterSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/SQLConfSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/SharedStateSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/internal/VariableSubstitutionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/jdbc/JDBCSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/BucketedReadSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/BucketedWriteSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/DDLSourceLoadSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/DataSourceTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/ExternalCommandRunnerSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/FilteredScanSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/FiltersSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/InsertSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/PartitionedWriteSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/PathOptionSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/PrunedScanSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/SaveLoadSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/TableScanSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/sources/fakeExternalSources.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/DeprecatedStreamingAggregationSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/FileStreamStressSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/MemorySourceStressSuite.scala (89%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/OffsetSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamTest.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala (97%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingJoinSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala (92%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingQuerySuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/StreamingSymmetricHashJoinHelperSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/ui/StreamingQueryPageSuite.scala (96%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/ui/StreamingQueryStatusListenerSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/ui/UISeleniumSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/ui/UIUtilsSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/util/BlockOnStopSource.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/util/BlockingSource.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/util/MockSourceProvider.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/streaming/util/StreamManualClock.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/GenericFlatSpecSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/GenericFunSpecSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/GenericWordSpecSuite.scala (91%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/ProcessTestUtils.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/SQLTestData.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/SQLTestUtils.scala (99%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/SharedSparkSession.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/TestSQLContext.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/test/TestSparkSessionSuite.scala (100%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/travis/TravisDataFrameAggregateSuite.scala (95%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/travis/TravisDataFrameJoinSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/util/DataFrameCallbackSuite.scala (98%) rename native-sql-engine/core/src/test/{scala => scala-fixme}/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala (100%) diff --git a/.github/workflows/report_ram_log.yml b/.github/workflows/report_ram_log.yml deleted file mode 100644 index 3067e2e73..000000000 --- a/.github/workflows/report_ram_log.yml +++ /dev/null @@ -1,65 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -name: Report RAM Log - -on: - workflow_run: - workflows: ["Native SQL Engine TPC-H Suite"] - types: - - completed - -jobs: - comment-on-pr: - if: ${{ github.event.workflow_run.conclusion == 'success' }} - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v2 - - name: Set up JDK 1.8 - uses: actions/setup-java@v1 - with: - java-version: 1.8 - - name: Download log - uses: dawidd6/action-download-artifact@v2 - with: - workflow: tpch.yml - run_id: ${{ github.event.workflow_run.id }} - name: comment_content - path: /tmp/ - - name: Download previous event payload - uses: dawidd6/action-download-artifact@v2 - with: - workflow: tpch.yml - run_id: ${{ github.event.workflow_run.id }} - name: pr_event - path: /tmp/ - - name: Install OAP optimized Arrow - run: | - cd /tmp - git clone -b arrow-3.0.0-oap https://github.com/oap-project/arrow.git - cd arrow/java - mvn clean install -B -P arrow-jni -am -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -Darrow.cpp.build.dir=/tmp/arrow/cpp/build/release/ -DskipTests -Dcheckstyle.skip - - name: Run Maven tests - run: | - mvn test -B -pl native-sql-engine/core/ -am -DmembersOnlySuites=com.intel.oap.tpc.h -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DtagsToInclude=com.intel.oap.tags.CommentOnContextPR -Dexec.skip=true - env: - MAVEN_OPTS: "-Xmx2048m" - COMMENT_CONTENT_PATH: "/tmp/comment.md" - PREVIOUS_EVENT_PATH: "/tmp/event.json" - GITHUB_TOKEN: ${{ github.token }} - ENABLE_TPCH_TESTS: "true" - diff --git a/.github/workflows/tpch.yml b/.github/workflows/tpch.yml index d0f655dbb..a66f00ee0 100644 --- a/.github/workflows/tpch.yml +++ b/.github/workflows/tpch.yml @@ -18,14 +18,23 @@ name: Native SQL Engine TPC-H Suite on: - pull_request + issue_comment: + types: [created, edited] jobs: ram-usage-test: - if: ${{ contains(github.event.pull_request.labels.*.name, 'RAM Report') }} + if: ${{ github.event.issue.pull_request && startsWith(github.event.comment.body, '@github-actions ram-usage-test') }} runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 + - name: Checkout Pull Request + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + run: | + PR_URL="${{ github.event.issue.pull_request.url }}" + PR_NUM=${PR_URL##*/} + echo "Checking out from PR #$PR_NUM based on URL: $PR_URL" + hub pr checkout $PR_NUM - name: Set up JDK 1.8 uses: actions/setup-java@v1 with: @@ -42,7 +51,7 @@ jobs: run: | cd /tmp git clone https://github.com/oap-project/arrow.git - cd arrow && git checkout arrow-3.0.0-oap && cd cpp + cd arrow && git checkout arrow-4.0.0-oap && cd cpp mkdir build && cd build cmake .. -DARROW_JNI=ON -DARROW_GANDIVA_JAVA=ON -DARROW_GANDIVA=ON -DARROW_PARQUET=ON -DARROW_CSV=ON -DARROW_HDFS=ON -DARROW_FILESYSTEM=ON -DARROW_WITH_SNAPPY=ON -DARROW_JSON=ON -DARROW_DATASET=ON -DARROW_WITH_LZ4=ON -DARROW_JEMALLOC=OFF && make -j2 sudo make install @@ -50,7 +59,7 @@ jobs: mvn clean install -B -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -P arrow-jni -am -Darrow.cpp.build.dir=/tmp/arrow/cpp/build/release/ -DskipTests -Dcheckstyle.skip - name: Run Maven tests - BHJ run: | - mvn test -B -pl native-sql-engine/core/ -am -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DmembersOnlySuites=com.intel.oap.tpc.h -DtagsToInclude=com.intel.oap.tags.BroadcastHashJoinMode -DargLine="-Xmx1G -XX:MaxDirectMemorySize=500M -Dio.netty.allocator.numDirectArena=1" + mvn test -B -P full-scala-compiler -Dbuild_arrow=OFF -pl native-sql-engine/core/ -am -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DmembersOnlySuites=com.intel.oap.tpc.h -DtagsToInclude=com.intel.oap.tags.BroadcastHashJoinMode -DargLine="-Xmx1G -XX:MaxDirectMemorySize=500M -Dio.netty.allocator.numDirectArena=1" env: MALLOC_ARENA_MAX: "4" MAVEN_OPTS: "-Xmx1G" @@ -59,7 +68,7 @@ jobs: ENABLE_TPCH_TESTS: "true" - name: Run Maven tests - SMJ run: | - mvn test -B -pl native-sql-engine/core/ -am -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DmembersOnlySuites=com.intel.oap.tpc.h -DtagsToInclude=com.intel.oap.tags.SortMergeJoinMode -DargLine="-Xmx1G -XX:MaxDirectMemorySize=500M -Dio.netty.allocator.numDirectArena=1" + mvn test -B -P full-scala-compiler -Dbuild_arrow=OFF -pl native-sql-engine/core/ -am -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DmembersOnlySuites=com.intel.oap.tpc.h -DtagsToInclude=com.intel.oap.tags.SortMergeJoinMode -DargLine="-Xmx1G -XX:MaxDirectMemorySize=500M -Dio.netty.allocator.numDirectArena=1" env: MALLOC_ARENA_MAX: "4" MAVEN_OPTS: "-Xmx1G" @@ -69,14 +78,12 @@ jobs: - run: | cml-publish /tmp/comment_image_1.png --md > /tmp/comment.md cml-publish /tmp/comment_image_2.png --md >> /tmp/comment.md - - run: echo "::set-output name=event_path::${GITHUB_EVENT_PATH}" - id: output-envs - - uses: actions/upload-artifact@v2 - with: - name: comment_content - path: /tmp/comment.md - - uses: actions/upload-artifact@v2 - with: - name: pr_event - path: ${{steps.output-envs.outputs.event_path}} - + - name: Run Maven tests - Report + run: | + mvn test -B -P full-scala-compiler -Dbuild_arrow=OFF -Dbuild_protobuf=OFF -pl native-sql-engine/core/ -am -DmembersOnlySuites=com.intel.oap.tpc.h -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn -DtagsToInclude=com.intel.oap.tags.CommentOnContextPR -Dexec.skip=true + env: + PR_URL: ${{ github.event.issue.pull_request.url }} + MAVEN_OPTS: "-Xmx1G" + COMMENT_CONTENT_PATH: "/tmp/comment.md" + GITHUB_TOKEN: ${{ github.token }} + ENABLE_TPCH_TESTS: "true" diff --git a/.github/workflows/unittests.yml b/.github/workflows/unittests.yml index efa405d60..797c5cfc7 100644 --- a/.github/workflows/unittests.yml +++ b/.github/workflows/unittests.yml @@ -60,6 +60,7 @@ jobs: ctest -R scala-unit-test: + if: ${{ github.event.issue.pull_request && startsWith(github.event.comment.body, '@github-actions scala-unit-test') }} runs-on: ubuntu-latest steps: - uses: actions/checkout@v2 @@ -82,8 +83,8 @@ jobs: - name: Install Spark run: | cd /tmp - wget http://archive.apache.org/dist/spark/spark-3.0.0/spark-3.0.0-bin-hadoop2.7.tgz - tar -xf spark-3.0.0-bin-hadoop2.7.tgz + wget http://archive.apache.org/dist/spark/spark-3.0.2/spark-3.0.2-bin-hadoop2.7.tgz + tar -xf spark-3.0.2-bin-hadoop2.7.tgz - name: Install OAP optimized Arrow (C++ libs) run: | cd /tmp @@ -100,9 +101,9 @@ jobs: cd arrow-data-source mvn clean install -DskipTests -Dbuild_arrow=OFF cd .. - mvn clean package -am -pl native-sql-engine/core -DskipTests -Dbuild_arrow=OFF + mvn clean package -P full-scala-compiler -am -pl native-sql-engine/core -DskipTests -Dbuild_arrow=OFF cd native-sql-engine/core/ - mvn test -DmembersOnlySuites=org.apache.spark.sql.travis -am -DfailIfNoTests=false -Dexec.skip=true -DargLine="-Dspark.test.home=/tmp/spark-3.0.0-bin-hadoop2.7" &> log-file.log + mvn test -P full-scala-compiler -DmembersOnlySuites=org.apache.spark.sql.travis -am -DfailIfNoTests=false -Dexec.skip=true -DargLine="-Dspark.test.home=/tmp/spark-3.0.0-bin-hadoop2.7" &> log-file.log echo '#!/bin/bash' > grep.sh echo "module_tested=0; module_should_test=1; tests_total=0; while read -r line; do num=\$(echo \"\$line\" | grep -o -E '[0-9]+'); tests_total=\$((tests_total+num)); done <<<\"\$(grep \"Total number of tests run:\" log-file.log)\"; succeed_total=0; while read -r line; do [[ \$line =~ [^0-9]*([0-9]+)\, ]]; num=\${BASH_REMATCH[1]}; succeed_total=\$((succeed_total+num)); let module_tested++; done <<<\"\$(grep \"succeeded\" log-file.log)\"; if test \$tests_total -eq \$succeed_total -a \$module_tested -eq \$module_should_test; then echo \"All unit tests succeed\"; else echo \"Unit tests failed\"; exit 1; fi" >> grep.sh bash grep.sh diff --git a/arrow-data-source/CHANGELOG.md b/CHANGELOG.md similarity index 54% rename from arrow-data-source/CHANGELOG.md rename to CHANGELOG.md index 96f64b55b..86b48d454 100644 --- a/arrow-data-source/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,288 @@ # Change log -Generated on 2020-12-21 +Generated on 2021-04-29 + +## Release 1.1.0 +* [Native SQL Engine](#native-sql-engine) +* [SQL DS Cache](#sql-ds-cache) +* [OAP MLlib](#oap-mllib) +* [PMEM Spill](#pmem-spill) +* [PMEM Shuffle](#pmem-shuffle) +* [Remote Shuffle](#remote-shuffle) + +### Native SQL Engine + +#### Features +||| +|:---|:---| +|[#261](https://github.com/oap-project/native-sql-engine/issues/261)|ArrowDataSource: Add S3 Support| +|[#239](https://github.com/oap-project/native-sql-engine/issues/239)|Adopt ARROW-7011| +|[#62](https://github.com/oap-project/native-sql-engine/issues/62)|Support Arrow's Build from Source and Package dependency library in the jar| +|[#145](https://github.com/oap-project/native-sql-engine/issues/145)|Support decimal in columnar window| +|[#31](https://github.com/oap-project/native-sql-engine/issues/31)|Decimal data type support| +|[#128](https://github.com/oap-project/native-sql-engine/issues/128)|Support Decimal in Aggregate| +|[#130](https://github.com/oap-project/native-sql-engine/issues/130)|Support decimal in project| +|[#134](https://github.com/oap-project/native-sql-engine/issues/134)|Update input metrics during reading| +|[#120](https://github.com/oap-project/native-sql-engine/issues/120)|Columnar window: Reduce peak memory usage and fix performance issues| +|[#108](https://github.com/oap-project/native-sql-engine/issues/108)|Add end-to-end test suite against TPC-DS| +|[#68](https://github.com/oap-project/native-sql-engine/issues/68)|Adaptive compression select in Shuffle.| +|[#97](https://github.com/oap-project/native-sql-engine/issues/97)|optimize null check in codegen sort| +|[#29](https://github.com/oap-project/native-sql-engine/issues/29)|Support mutiple-key sort without codegen| +|[#75](https://github.com/oap-project/native-sql-engine/issues/75)|Support HashAggregate in ColumnarWSCG| +|[#73](https://github.com/oap-project/native-sql-engine/issues/73)|improve columnar SMJ| +|[#51](https://github.com/oap-project/native-sql-engine/issues/51)|Decimal fallback| +|[#38](https://github.com/oap-project/native-sql-engine/issues/38)|Supporting expression as join keys in columnar SMJ| +|[#27](https://github.com/oap-project/native-sql-engine/issues/27)|Support REUSE exchange when DPP enabled| +|[#17](https://github.com/oap-project/native-sql-engine/issues/17)|ColumnarWSCG further optimization| + +#### Performance +||| +|:---|:---| +|[#194](https://github.com/oap-project/native-sql-engine/issues/194)|Arrow Parameters Update when compiling Arrow| +|[#136](https://github.com/oap-project/native-sql-engine/issues/136)|upgrade to arrow 3.0| +|[#103](https://github.com/oap-project/native-sql-engine/issues/103)|reduce codegen in multiple-key sort| +|[#90](https://github.com/oap-project/native-sql-engine/issues/90)|Refine HashAggregate to do everything in CPP| + +#### Bugs Fixed +||| +|:---|:---| +|[#278](https://github.com/oap-project/native-sql-engine/issues/278)|fix arrow dep in 1.1 branch| +|[#265](https://github.com/oap-project/native-sql-engine/issues/265)|TPC-DS Q67 failed with memmove exception in native split code.| +|[#280](https://github.com/oap-project/native-sql-engine/issues/280)|CMake version check| +|[#241](https://github.com/oap-project/native-sql-engine/issues/241)|TPC-DS q67 failed for XXH3_hashLong_64b_withSecret.constprop.0+0x180| +|[#262](https://github.com/oap-project/native-sql-engine/issues/262)|q18 has different digits compared with vanilla spark| +|[#196](https://github.com/oap-project/native-sql-engine/issues/196)|clean up options for native sql engine| +|[#224](https://github.com/oap-project/native-sql-engine/issues/224)|update 3rd party libs| +|[#227](https://github.com/oap-project/native-sql-engine/issues/227)|fix vulnerabilities from klockwork| +|[#237](https://github.com/oap-project/native-sql-engine/issues/237)|Add ARROW_CSV=ON to default C++ build commands| +|[#229](https://github.com/oap-project/native-sql-engine/issues/229)|Fix the deprecated code warning in shuffle_split_test| +|[#119](https://github.com/oap-project/native-sql-engine/issues/119)|consolidate batch size| +|[#217](https://github.com/oap-project/native-sql-engine/issues/217)|TPC-H query20 result not correct when use decimal dataset| +|[#211](https://github.com/oap-project/native-sql-engine/issues/211)|IndexOutOfBoundsException during running TPC-DS Q2| +|[#167](https://github.com/oap-project/native-sql-engine/issues/167)|Cannot successfully run q.14a.sql and q14b.sql when using double format for TPC-DS workload.| +|[#191](https://github.com/oap-project/native-sql-engine/issues/191)|libarrow.so and libgandiva.so not copy into the tmp directory| +|[#179](https://github.com/oap-project/native-sql-engine/issues/179)|Unable to find Arrow headers during build| +|[#153](https://github.com/oap-project/native-sql-engine/issues/153)|Fix incorrect queries after enabled Decimal| +|[#173](https://github.com/oap-project/native-sql-engine/issues/173)|fix the incorrect result of q69| +|[#48](https://github.com/oap-project/native-sql-engine/issues/48)|unit tests for c++ are broken| +|[#101](https://github.com/oap-project/native-sql-engine/issues/101)|ColumnarWindow: Remove obsolete debug code| +|[#100](https://github.com/oap-project/native-sql-engine/issues/100)|Incorrect result in Q45 w/ v2 bhj threshold is 10MB sf500| +|[#81](https://github.com/oap-project/native-sql-engine/issues/81)|Some ArrowVectorWriter implementations doesn't implement setNulls method| +|[#82](https://github.com/oap-project/native-sql-engine/issues/82)|Incorrect result in TPCDS Q72 SF1536| +|[#70](https://github.com/oap-project/native-sql-engine/issues/70)|Duplicate IsNull check in codegen sort| +|[#64](https://github.com/oap-project/native-sql-engine/issues/64)|Memleak in sort when SMJ is disabled| +|[#58](https://github.com/oap-project/native-sql-engine/issues/58)|Issues when running tpcds with DPP enabled and AQE disabled | +|[#52](https://github.com/oap-project/native-sql-engine/issues/52)|memory leakage in columnar SMJ| +|[#53](https://github.com/oap-project/native-sql-engine/issues/53)|Q24a/Q24b SHJ tail task took about 50 secs in SF1500| +|[#42](https://github.com/oap-project/native-sql-engine/issues/42)|reduce columnar sort memory footprint| +|[#40](https://github.com/oap-project/native-sql-engine/issues/40)|columnar sort codegen fallback to executor side| +|[#1](https://github.com/oap-project/native-sql-engine/issues/1)|columnar whole stage codegen failed due to empty results| +|[#23](https://github.com/oap-project/native-sql-engine/issues/23)|TPC-DS Q8 failed due to unsupported operation in columnar sortmergejoin| +|[#22](https://github.com/oap-project/native-sql-engine/issues/22)|TPC-DS Q95 failed due in columnar wscg| +|[#4](https://github.com/oap-project/native-sql-engine/issues/4)|columnar BHJ failed on new memory pool| +|[#5](https://github.com/oap-project/native-sql-engine/issues/5)|columnar BHJ failed on partitioned table with prefercolumnar=false| + +#### PRs +||| +|:---|:---| +|[#288](https://github.com/oap-project/native-sql-engine/pull/288)|[NSE-119] clean up on comments| +|[#282](https://github.com/oap-project/native-sql-engine/pull/282)|[NSE-280]fix cmake version check| +|[#281](https://github.com/oap-project/native-sql-engine/pull/281)|[NSE-280] bump cmake to 3.16| +|[#279](https://github.com/oap-project/native-sql-engine/pull/279)|[NSE-278]fix arrow dep in 1.1 branch| +|[#268](https://github.com/oap-project/native-sql-engine/pull/268)|[NSE-186] backport to 1.1 branch| +|[#266](https://github.com/oap-project/native-sql-engine/pull/266)|[NSE-265] Reserve enough memory before UnsafeAppend in builder| +|[#270](https://github.com/oap-project/native-sql-engine/pull/270)|[NSE-261] ArrowDataSource: Add S3 Support| +|[#263](https://github.com/oap-project/native-sql-engine/pull/263)|[NSE-262] fix remainer loss in decimal divide| +|[#215](https://github.com/oap-project/native-sql-engine/pull/215)|[NSE-196] clean up native sql options| +|[#231](https://github.com/oap-project/native-sql-engine/pull/231)|[NSE-176]Arrow install order issue| +|[#242](https://github.com/oap-project/native-sql-engine/pull/242)|[NSE-224] update third party code| +|[#240](https://github.com/oap-project/native-sql-engine/pull/240)|[NSE-239] Adopt ARROW-7011| +|[#238](https://github.com/oap-project/native-sql-engine/pull/238)|[NSE-237] Add ARROW_CSV=ON to default C++ build commands| +|[#230](https://github.com/oap-project/native-sql-engine/pull/230)|[NSE-229] Fix the deprecated code warning in shuffle_split_test| +|[#225](https://github.com/oap-project/native-sql-engine/pull/225)|[NSE-227]fix issues from codescan| +|[#219](https://github.com/oap-project/native-sql-engine/pull/219)|[NSE-217] fix missing decimal check| +|[#212](https://github.com/oap-project/native-sql-engine/pull/212)|[NSE-211] IndexOutOfBoundsException during running TPC-DS Q2| +|[#187](https://github.com/oap-project/native-sql-engine/pull/187)|[NSE-185] Avoid unnecessary copying when simply projecting on fields| +|[#195](https://github.com/oap-project/native-sql-engine/pull/195)|[NSE-194]Turn on several Arrow parameters| +|[#189](https://github.com/oap-project/native-sql-engine/pull/189)|[NSE-153] Following NSE-153, optimize fallback conditions for columnar window| +|[#192](https://github.com/oap-project/native-sql-engine/pull/192)|[NSE-191]Fix issue0191 for .so file copy to tmp.| +|[#181](https://github.com/oap-project/native-sql-engine/pull/181)|[NSE-179]Fix arrow include directory not include when using ARROW_ROOT| +|[#175](https://github.com/oap-project/native-sql-engine/pull/175)|[NSE-153] Fix window results| +|[#174](https://github.com/oap-project/native-sql-engine/pull/174)|[NSE-173] fix incorrect result of q69| +|[#172](https://github.com/oap-project/native-sql-engine/pull/172)|[NSE-62]Fixing issue0062 for package arrow dependencies in jar with refresh2| +|[#171](https://github.com/oap-project/native-sql-engine/pull/171)|[NSE-170]improve sort shuffle code| +|[#165](https://github.com/oap-project/native-sql-engine/pull/165)|[NSE-161] adding format check| +|[#166](https://github.com/oap-project/native-sql-engine/pull/166)|[NSE-130] support decimal round and abs| +|[#164](https://github.com/oap-project/native-sql-engine/pull/164)|[NSE-130] fix precision loss in divide w/ decimal type| +|[#159](https://github.com/oap-project/native-sql-engine/pull/159)|[NSE-31] fix SMJ divide with decimal| +|[#156](https://github.com/oap-project/native-sql-engine/pull/156)|[NSE-130] fix overflow and precision loss| +|[#152](https://github.com/oap-project/native-sql-engine/pull/152)|[NSE-86] Merge Arrow Data Source| +|[#154](https://github.com/oap-project/native-sql-engine/pull/154)|[NSE-153] Fix incorrect quries after enabled Decimal| +|[#151](https://github.com/oap-project/native-sql-engine/pull/151)|[NSE-145] Support decimal in columnar window| +|[#129](https://github.com/oap-project/native-sql-engine/pull/129)|[NSE-128]Support Decimal in Aggregate/HashJoin| +|[#131](https://github.com/oap-project/native-sql-engine/pull/131)|[NSE-130] support decimal in project| +|[#107](https://github.com/oap-project/native-sql-engine/pull/107)|[NSE-136]upgrade to arrow 3.0.0| +|[#135](https://github.com/oap-project/native-sql-engine/pull/135)|[NSE-134] Update input metrics during reading| +|[#121](https://github.com/oap-project/native-sql-engine/pull/121)|[NSE-120] Columnar window: Reduce peak memory usage and fix performance issues| +|[#112](https://github.com/oap-project/native-sql-engine/pull/112)|[NSE-97] optimize null check and refactor sort kernels| +|[#109](https://github.com/oap-project/native-sql-engine/pull/109)|[NSE-108] Add end-to-end test suite against TPC-DS| +|[#69](https://github.com/oap-project/native-sql-engine/pull/69)|[NSE-68][Shuffle] Adaptive compression select in Shuffle.| +|[#98](https://github.com/oap-project/native-sql-engine/pull/98)|[NSE-97] remove isnull when null count is zero| +|[#102](https://github.com/oap-project/native-sql-engine/pull/102)|[NSE-101] ColumnarWindow: Remove obsolete debug code| +|[#105](https://github.com/oap-project/native-sql-engine/pull/105)|[NSE-100]Fix an incorrect result error when using SHJ in Q45| +|[#91](https://github.com/oap-project/native-sql-engine/pull/91)|[NSE-90]Refactor HashAggregateExec and CPP kernels| +|[#79](https://github.com/oap-project/native-sql-engine/pull/79)|[NSE-81] add missing setNulls methods in ArrowWritableColumnVector| +|[#44](https://github.com/oap-project/native-sql-engine/pull/44)|[NSE-29]adding non-codegen framework for multiple-key sort| +|[#76](https://github.com/oap-project/native-sql-engine/pull/76)|[NSE-75]Support ColumnarHashAggregate in ColumnarWSCG| +|[#83](https://github.com/oap-project/native-sql-engine/pull/83)|[NSE-82] Fix Q72 SF1536 incorrect result| +|[#72](https://github.com/oap-project/native-sql-engine/pull/72)|[NSE-51] add more datatype fallback logic in columnar operators| +|[#60](https://github.com/oap-project/native-sql-engine/pull/60)|[NSE-48] fix c++ unit tests| +|[#50](https://github.com/oap-project/native-sql-engine/pull/50)|[NSE-45] BHJ memory leak| +|[#74](https://github.com/oap-project/native-sql-engine/pull/74)|[NSE-73]using data ref in multiple keys based SMJ| +|[#71](https://github.com/oap-project/native-sql-engine/pull/71)|[NSE-70] remove duplicate IsNull check in sort| +|[#65](https://github.com/oap-project/native-sql-engine/pull/65)|[NSE-64] fix memleak in sort when SMJ is disabled| +|[#59](https://github.com/oap-project/native-sql-engine/pull/59)|[NSE-58]Fix empty input issue when DPP enabled| +|[#7](https://github.com/oap-project/native-sql-engine/pull/7)|[OAP-1846][oap-native-sql] add more fallback logic | +|[#57](https://github.com/oap-project/native-sql-engine/pull/57)|[NSE-56]ColumnarSMJ: fallback on full outer join| +|[#55](https://github.com/oap-project/native-sql-engine/pull/55)|[NSE-52]Columnar SMJ: fix memory leak by closing stream batches properly| +|[#54](https://github.com/oap-project/native-sql-engine/pull/54)|[NSE-53]Partial fix Q24a/Q24b tail SHJ task materialization performance issue| +|[#47](https://github.com/oap-project/native-sql-engine/pull/47)|[NSE-17]TPCDS Q72 optimization| +|[#39](https://github.com/oap-project/native-sql-engine/pull/39)|[NSE-38]ColumnarSMJ: support expression as join keys| +|[#43](https://github.com/oap-project/native-sql-engine/pull/43)|[NSE-42] early release sort input| +|[#33](https://github.com/oap-project/native-sql-engine/pull/33)|[NSE-32] Use Spark managed spill in columnar shuffle| +|[#41](https://github.com/oap-project/native-sql-engine/pull/41)|[NSE-40] fixes driver failing to do sort codege| +|[#28](https://github.com/oap-project/native-sql-engine/pull/28)|[NSE-27]Reuse exchage to optimize DPP performance| +|[#36](https://github.com/oap-project/native-sql-engine/pull/36)|[NSE-1]fix columnar wscg on empty recordbatch| +|[#24](https://github.com/oap-project/native-sql-engine/pull/24)|[NSE-23]fix columnar SMJ fallback| +|[#26](https://github.com/oap-project/native-sql-engine/pull/26)|[NSE-22]Fix w/DPP issue when inside wscg smj both sides are smj| +|[#18](https://github.com/oap-project/native-sql-engine/pull/18)|[NSE-17] smjwscg optimization:| +|[#3](https://github.com/oap-project/native-sql-engine/pull/3)|[NSE-4]fix columnar BHJ on new memory pool| +|[#6](https://github.com/oap-project/native-sql-engine/pull/6)|[NSE-5][SCALA] Fix ColumnarBroadcastExchange didn't fallback issue w/ DPP| + + +### SQL DS Cache + +#### Features +||| +|:---|:---| +|[#36](https://github.com/oap-project/sql-ds-cache/issues/36)|HCFS doc for Spark| +|[#38](https://github.com/oap-project/sql-ds-cache/issues/38)|update Plasma dependency for Plasma-based-cache module| +|[#14](https://github.com/oap-project/sql-ds-cache/issues/14)|Add HCFS module| +|[#17](https://github.com/oap-project/sql-ds-cache/issues/17)|replace arrow-plasma dependency for hcfs module| + +#### Bugs Fixed +||| +|:---|:---| +|[#62](https://github.com/oap-project/sql-ds-cache/issues/62)|Upgrade hadoop dependencies in HCFS| + +#### PRs +||| +|:---|:---| +|[#83](https://github.com/oap-project/sql-ds-cache/pull/83)|[SQL-DS-CACHE-82][SDLe]Upgrade Jetty version| +|[#77](https://github.com/oap-project/sql-ds-cache/pull/77)|[SQL-DS-CACHE-62][POAE7-984] upgrade hadoop version to 3.3.0| +|[#56](https://github.com/oap-project/sql-ds-cache/pull/56)|[SQL-DS-CACHE-47]Add plasma native get timeout| +|[#37](https://github.com/oap-project/sql-ds-cache/pull/37)|[SQL-DS-CACHE-36][POAE7-898]HCFS docs for OAP 1.1| +|[#39](https://github.com/oap-project/sql-ds-cache/pull/39)|[SQL-DS-CACHE-38][POAE7-892]update Plasma dependency| +|[#18](https://github.com/oap-project/sql-ds-cache/pull/18)|[SQL-DS-CACHE-17][POAE7-905]replace intel-arrow with apache-arrow v3.0.0| +|[#13](https://github.com/oap-project/sql-ds-cache/pull/13)|[SQL-DS-CACHE-14][POAE7-847] Port HCFS to OAP| +|[#16](https://github.com/oap-project/sql-ds-cache/pull/16)|[SQL-DS-CACHE-15][POAE7-869]Refactor original code to make it a sub-module| + + +### OAP MLlib + +#### Features +||| +|:---|:---| +|[#35](https://github.com/oap-project/oap-mllib/issues/35)|Restrict printNumericTable to first 10 eigenvalues with first 20 dimensions| +|[#33](https://github.com/oap-project/oap-mllib/issues/33)|Optimize oneCCL port detecting| +|[#28](https://github.com/oap-project/oap-mllib/issues/28)|Use getifaddrs to get host ips for oneCCL kvs| +|[#12](https://github.com/oap-project/oap-mllib/issues/12)|Improve CI and add pseudo cluster testing| +|[#31](https://github.com/oap-project/oap-mllib/issues/31)|Print time duration for each PCA step| +|[#13](https://github.com/oap-project/oap-mllib/issues/13)|Add ALS with new oneCCL APIs| +|[#18](https://github.com/oap-project/oap-mllib/issues/18)|Auto detect KVS port for oneCCL to avoid port conflict| +|[#10](https://github.com/oap-project/oap-mllib/issues/10)|Porting Kmeans and PCA to new oneCCL API| + +#### Bugs Fixed +||| +|:---|:---| +|[#43](https://github.com/oap-project/oap-mllib/issues/43)|[Release] Error when installing intel-oneapi-dal-devel-2021.1.1 intel-oneapi-tbb-devel-2021.1.1| +|[#46](https://github.com/oap-project/oap-mllib/issues/46)|[Release] Meet hang issue when running PCA algorithm.| +|[#48](https://github.com/oap-project/oap-mllib/issues/48)|[Release] No performance benefit when using Intel-MLlib to run ALS algorithm.| +|[#25](https://github.com/oap-project/oap-mllib/issues/25)|Fix oneCCL KVS port auto detect and improve logging| + +#### PRs +||| +|:---|:---| +|[#51](https://github.com/oap-project/oap-mllib/pull/51)|[ML-50] Merge #47 and prepare for OAP 1.1| +|[#49](https://github.com/oap-project/oap-mllib/pull/49)|Revert "[ML-41] Revert to old oneCCL and Prepare for OAP 1.1"| +|[#47](https://github.com/oap-project/oap-mllib/pull/47)|[ML-44] [PIP] Update to oneAPI 2021.2 and Rework examples for validation| +|[#40](https://github.com/oap-project/oap-mllib/pull/40)|[ML-41] Revert to old oneCCL and Prepare for OAP 1.1| +|[#36](https://github.com/oap-project/oap-mllib/pull/36)|[ML-35] Restrict printNumericTable to first 10 eigenvalues with first 20 dimensions| +|[#34](https://github.com/oap-project/oap-mllib/pull/34)|[ML-33] Optimize oneCCL port detecting| +|[#20](https://github.com/oap-project/oap-mllib/pull/20)|[ML-12] Improve CI and add pseudo cluster testing| +|[#32](https://github.com/oap-project/oap-mllib/pull/32)|[ML-31] Print time duration for each PCA step| +|[#14](https://github.com/oap-project/oap-mllib/pull/14)|[ML-13] Add ALS with new oneCCL APIs| +|[#24](https://github.com/oap-project/oap-mllib/pull/24)|[ML-25] Fix oneCCL KVS port auto detect and improve logging| +|[#19](https://github.com/oap-project/oap-mllib/pull/19)|[ML-18] Auto detect KVS port for oneCCL to avoid port conflict| + + +### PMEM Spill + +#### Bugs Fixed +||| +|:---|:---| +|[#22](https://github.com/oap-project/pmem-spill/issues/22)|[SDLe][Snyk]Upgrade Jetty version to fix vulnerability scanned by Snyk| +|[#13](https://github.com/oap-project/pmem-spill/issues/13)|The compiled code failed because the variable name was not changed| + +#### PRs +||| +|:---|:---| +|[#27](https://github.com/oap-project/pmem-spill/pull/27)|[PMEM-SPILL-22][SDLe]Upgrade Jetty version| +|[#21](https://github.com/oap-project/pmem-spill/pull/21)|[POAE7-961] fix null pointer issue when offheap enabled.| +|[#18](https://github.com/oap-project/pmem-spill/pull/18)|[POAE7-858] disable RDD cache related PMem intialization as default and add PMem related logic in SparkEnv| +|[#19](https://github.com/oap-project/pmem-spill/pull/19)|[PMEM-SPILL-20][POAE7-912]add vanilla SparkEnv.scala for future update| +|[#15](https://github.com/oap-project/pmem-spill/pull/15)|[POAE7-858] port memory extension options to OAP 1.1| +|[#12](https://github.com/oap-project/pmem-spill/pull/12)|Change the variable name so that the passed parameters are correct| +|[#10](https://github.com/oap-project/pmem-spill/pull/10)|Fixing one pmem path on AppDirect mode may cause the pmem initialization path to be empty Path| + + +### PMEM Shuffle + +#### Features +||| +|:---|:---| +|[#7](https://github.com/oap-project/pmem-shuffle/issues/7)|Enable running in fsdax mode| + +#### Bugs Fixed +||| +|:---|:---| +|[#10](https://github.com/oap-project/pmem-shuffle/issues/10)|[pmem-shuffle] There are potential issues reported by Klockwork. | + +#### PRs +||| +|:---|:---| +|[#13](https://github.com/oap-project/pmem-shuffle/pull/13)|[PMEM-SHUFFLE-10] Fix potential issues reported by klockwork for branch 1.1. | +|[#6](https://github.com/oap-project/pmem-shuffle/pull/6)|[PMEM-SHUFFLE-7] enable fsdax mode in pmem-shuffle| + + +### Remote-Shuffle + +#### Features +||| +|:---|:---| +|[#6](https://github.com/oap-project/remote-shuffle/issues/6)|refactor shuffle-daos by abstracting shuffle IO for supporting both synchronous and asynchronous DAOS Object API| +|[#4](https://github.com/oap-project/remote-shuffle/issues/4)|check-in remote shuffle based on DAOS Object API| + +#### Bugs Fixed +||| +|:---|:---| +|[#12](https://github.com/oap-project/remote-shuffle/issues/12)|[SDLe][Snyk]Upgrade org.mock-server:mockserver-netty to fix vulnerability scanned by Snyk| + +#### PRs +||| +|:---|:---| +|[#13](https://github.com/oap-project/remote-shuffle/pull/13)|[REMOTE-SHUFFLE-12][SDle][Snyk]Upgrade org.mock-server:mockserver-net…| +|[#5](https://github.com/oap-project/remote-shuffle/pull/5)|check-in remote shuffle based on DAOS Object API| + ## Release 1.0.0 diff --git a/LICENSE b/LICENSE new file mode 100644 index 000000000..47eb519be --- /dev/null +++ b/LICENSE @@ -0,0 +1,1957 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- + +src/plasma/fling.cc and src/plasma/fling.h: Apache 2.0 + +Copyright 2013 Sharvil Nanavati + +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. + +-------------------------------------------------------------------------------- + +src/plasma/thirdparty/dlmalloc.c: CC0 + +This is a version (aka dlmalloc) of malloc/free/realloc written by +Doug Lea and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ Send questions, +comments, complaints, performance data, etc to dl@cs.oswego.edu + +-------------------------------------------------------------------------------- + +src/plasma/common.cc (some portions) + +Copyright (c) Austin Appleby (aappleby (AT) gmail) + +Some portions of this file are derived from code in the MurmurHash project + +All code is released to the public domain. For business purposes, Murmurhash is +under the MIT license. + +https://sites.google.com/site/murmurhash/ + +-------------------------------------------------------------------------------- + +src/arrow/util (some portions): Apache 2.0, and 3-clause BSD + +Some portions of this module are derived from code in the Chromium project, +copyright (c) Google inc and (c) The Chromium Authors and licensed under the +Apache 2.0 License or the under the 3-clause BSD license: + + Copyright (c) 2013 The Chromium Authors. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from Daniel Lemire's FrameOfReference project. + +https://github.com/lemire/FrameOfReference/blob/6ccaf9e97160f9a3b299e23a8ef739e711ef0c71/src/bpacking.cpp + +Copyright: 2013 Daniel Lemire +Home page: http://lemire.me/en/ +Project page: https://github.com/lemire/FrameOfReference +License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the TensorFlow project + +Copyright 2015 The TensorFlow Authors. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the NumPy project. + +https://github.com/numpy/numpy/blob/e1f191c46f2eebd6cb892a4bfe14d9dd43a06c4e/numpy/core/src/multiarray/multiarraymodule.c#L2910 + +https://github.com/numpy/numpy/blob/68fd82271b9ea5a9e50d4e761061dfcca851382a/numpy/core/src/multiarray/datetime.c + +Copyright (c) 2005-2017, NumPy Developers. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the NumPy Developers nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the Boost project + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from the FlatBuffers project + +Copyright 2014 Google Inc. + +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. + +-------------------------------------------------------------------------------- + +This project includes code from the tslib project + +Copyright 2015 Microsoft Corporation. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the jemalloc project + +https://github.com/jemalloc/jemalloc + +Copyright (C) 2002-2017 Jason Evans . +All rights reserved. +Copyright (C) 2007-2012 Mozilla Foundation. All rights reserved. +Copyright (C) 2009-2017 Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: +1. Redistributions of source code must retain the above copyright notice(s), + this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice(s), + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER(S) ``AS IS'' AND ANY EXPRESS +OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO +EVENT SHALL THE COPYRIGHT HOLDER(S) BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE +OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF +ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------------------- + +This project includes code from the Go project, BSD 3-clause license + PATENTS +weak patent termination clause +(https://github.com/golang/go/blob/master/PATENTS). + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the hs2client + +https://github.com/cloudera/hs2client + +Copyright 2016 Cloudera Inc. + +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. + +-------------------------------------------------------------------------------- + +The script ci/scripts/util_wait_for_it.sh has the following license + +Copyright (c) 2016 Giles Hall + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The script r/configure has the following license (MIT) + +Copyright (c) 2017, Jeroen Ooms and Jim Hester + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +cpp/src/arrow/util/logging.cc, cpp/src/arrow/util/logging.h and +cpp/src/arrow/util/logging-test.cc are adapted from +Ray Project (https://github.com/ray-project/ray) (Apache 2.0). + +Copyright (c) 2016 Ray Project (https://github.com/ray-project/ray) + +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. + +-------------------------------------------------------------------------------- +The files cpp/src/arrow/vendored/datetime/date.h, cpp/src/arrow/vendored/datetime/tz.h, +cpp/src/arrow/vendored/datetime/tz_private.h, cpp/src/arrow/vendored/datetime/ios.h, +cpp/src/arrow/vendored/datetime/tz.cpp are adapted from +Howard Hinnant's date library (https://github.com/HowardHinnant/date) +It is licensed under MIT license. + +The MIT License (MIT) +Copyright (c) 2015, 2016, 2017 Howard Hinnant +Copyright (c) 2016 Adrian Colomitchi +Copyright (c) 2017 Florian Dang +Copyright (c) 2017 Paul Thompson +Copyright (c) 2018 Tomasz Kamiński + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/util/utf8.h includes code adapted from the page + https://bjoern.hoehrmann.de/utf-8/decoder/dfa/ +with the following license (MIT) + +Copyright (c) 2008-2009 Bjoern Hoehrmann + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/string_view.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/variant.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/xxhash/ have the following license +(BSD 2-Clause License) + +xxHash Library +Copyright (c) 2012-2014, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +You can contact the author at : +- xxHash homepage: http://www.xxhash.com +- xxHash source repository : https://github.com/Cyan4973/xxHash + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/double-conversion/ have the following license +(BSD 3-Clause License) + +Copyright 2006-2011, the V8 project authors. All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/uriparser/ have the following license +(BSD 3-Clause License) + +uriparser - RFC 3986 URI parsing library + +Copyright (C) 2007, Weijia Song +Copyright (C) 2007, Sebastian Pipping +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + * Redistributions of source code must retain the above + copyright notice, this list of conditions and the following + disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials + provided with the distribution. + + * Neither the name of the nor the names of its + contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE +COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED +OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files under dev/tasks/conda-recipes have the following license + +BSD 3-clause license +Copyright (c) 2015-2018, conda-forge +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR +TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/utf8cpp/ have the following license + +Copyright 2006 Nemanja Trifunovic + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from Apache Kudu. + + * cpp/cmake_modules/CompilerInfo.cmake is based on Kudu's cmake_modules/CompilerInfo.cmake + +Copyright: 2016 The Apache Software Foundation. +Home page: https://kudu.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Impala (incubating), formerly +Impala. The Impala code and rights were donated to the ASF as part of the +Incubator process after the initial code imports into Apache Parquet. + +Copyright: 2012 Cloudera, Inc. +Copyright: 2016 The Apache Software Foundation. +Home page: http://impala.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Aurora. + +* dev/release/{release,changelog,release-candidate} are based on the scripts from + Apache Aurora + +Copyright: 2016 The Apache Software Foundation. +Home page: https://aurora.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the Google styleguide. + +* cpp/build-support/cpplint.py is based on the scripts from the Google styleguide. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/styleguide +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from Snappy. + +* cpp/cmake_modules/{SnappyCMakeLists.txt,SnappyConfig.h} are based on code + from Google's Snappy project. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/snappy +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from the manylinux project. + +* python/manylinux1/scripts/{build_python.sh,python-tag-abi-tag.py, + requirements.txt} are based on code from the manylinux project. + +Copyright: 2016 manylinux +Homepage: https://github.com/pypa/manylinux +License: The MIT License (MIT) + +-------------------------------------------------------------------------------- + +This project includes code from the cymove project: + +* python/pyarrow/includes/common.pxd includes code from the cymove project + +The MIT License (MIT) +Copyright (c) 2019 Omer Ozarslan + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE +OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The projects includes code from the Ursabot project under the dev/archery +directory. + +License: BSD 2-Clause + +Copyright 2019 RStudio, Inc. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project include code from CMake. + +* cpp/cmake_modules/FindGTest.cmake is based on code from CMake. + +Copyright: Copyright 2000-2019 Kitware, Inc. and Contributors +Homepage: https://gitlab.kitware.com/cmake/cmake +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project include code from mingw-w64. + +* cpp/src/arrow/util/cpu-info.cc has a polyfill for mingw-w64 < 5 + +Copyright (c) 2009 - 2013 by the mingw-w64 project +Homepage: https://mingw-w64.org +License: Zope Public License (ZPL) Version 2.1. + +--------------------------------------------------------------------------------- + +This project include code from Google's Asylo project. + +* cpp/src/arrow/result.h is based on status_or.h + +Copyright (c) Copyright 2017 Asylo authors +Homepage: https://asylo.dev/ +License: Apache 2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Google's protobuf project + +* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN + +Copyright 2008 Google Inc. All rights reserved. +Homepage: https://developers.google.com/protocol-buffers/ +License: + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +3rdparty dependency LLVM is statically linked in certain binary +distributions. LLVM has the following license: + +============================================================================== +LLVM Release License +============================================================================== +University of Illinois/NCSA +Open Source License + +Copyright (c) 2003-2018 University of Illinois at Urbana-Champaign. +All rights reserved. + +Developed by: + + LLVM Team + + University of Illinois at Urbana-Champaign + + http://llvm.org + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal with +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimers. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimers in the + documentation and/or other materials provided with the distribution. + + * Neither the names of the LLVM Team, University of Illinois at + Urbana-Champaign, nor the names of its contributors may be used to + endorse or promote products derived from this Software without specific + prior written permission. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS +FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +CONTRIBUTORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS WITH THE +SOFTWARE. + +============================================================================== +Copyrights and Licenses for Third Party Software Distributed with LLVM: +============================================================================== +The LLVM software contains code written by third parties. Such software will +have its own individual LICENSE.TXT file in the directory in which it appears. +This file will describe the copyrights, license, and restrictions which apply +to that code. + +The disclaimer of warranty in the University of Illinois Open Source License +applies to all code in the LLVM Distribution, and nothing in any of the +other licenses gives permission to use the names of the LLVM Team or the +University of Illinois to endorse or promote products derived from this +Software. + +The following pieces of software have additional or alternate copyrights, +licenses, and/or restrictions: + +Program Directory +------- --------- +Google Test llvm/utils/unittest/googletest +OpenBSD regex llvm/lib/Support/{reg*, COPYRIGHT.regex} +pyyaml tests llvm/test/YAMLParser/{*.data, LICENSE.TXT} +ARM contributions llvm/lib/Target/ARM/LICENSE.TXT +md5 contributions llvm/lib/Support/MD5.cpp llvm/include/llvm/Support/MD5.h + +-------------------------------------------------------------------------------- + +3rdparty dependency gRPC is statically linked in certain binary +distributions, like the python wheels. gRPC has the following license: + +Copyright 2014 gRPC authors. + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache Thrift is statically linked in certain binary +distributions, like the python wheels. Apache Thrift has the following license: + +Apache Thrift +Copyright (C) 2006 - 2019, The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache ORC is statically linked in certain binary +distributions, like the python wheels. Apache ORC has the following license: + +Apache ORC +Copyright 2013-2019 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by Hewlett-Packard: +(c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency zstd is statically linked in certain binary +distributions, like the python wheels. ZSTD has the following license: + +BSD License + +For Zstandard software + +Copyright (c) 2016-present, Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + * Neither the name Facebook nor the names of its contributors may be used to + endorse or promote products derived from this software without specific + prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency lz4 is statically linked in certain binary +distributions, like the python wheels. lz4 has the following license: + +LZ4 Library +Copyright (c) 2011-2016, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency Brotli is statically linked in certain binary +distributions, like the python wheels. Brotli has the following license: + +Copyright (c) 2009, 2010, 2013-2016 by the Brotli Authors. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency rapidjson is statically linked in certain binary +distributions, like the python wheels. rapidjson and its dependencies have the +following licenses: + +Tencent is pleased to support the open source community by making RapidJSON +available. + +Copyright (C) 2015 THL A29 Limited, a Tencent company, and Milo Yip. +All rights reserved. + +If you have downloaded a copy of the RapidJSON binary from Tencent, please note +that the RapidJSON binary is licensed under the MIT License. +If you have downloaded a copy of the RapidJSON source code from Tencent, please +note that RapidJSON source code is licensed under the MIT License, except for +the third-party components listed below which are subject to different license +terms. Your integration of RapidJSON into your own projects may require +compliance with the MIT License, as well as the other licenses applicable to +the third-party components included within RapidJSON. To avoid the problematic +JSON license in your own projects, it's sufficient to exclude the +bin/jsonchecker/ directory, as it's the only code under the JSON license. +A copy of the MIT License is included in this file. + +Other dependencies and licenses: + + Open Source Software Licensed Under the BSD License: + -------------------------------------------------------------------- + + The msinttypes r29 + Copyright (c) 2006-2013 Alexander Chemeris + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE REGENTS AND CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + + Open Source Software Licensed Under the JSON License: + -------------------------------------------------------------------- + + json.org + Copyright (c) 2002 JSON.org + All Rights Reserved. + + JSON_checker + Copyright (c) 2002 JSON.org + All Rights Reserved. + + + Terms of the JSON License: + --------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + + The Software shall be used for Good, not Evil. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + + + Terms of the MIT License: + -------------------------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included + in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency snappy is statically linked in certain binary +distributions, like the python wheels. snappy has the following license: + +Copyright 2011, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of Google Inc. nor the names of its contributors may be + used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +=== + +Some of the benchmark data in testdata/ is licensed differently: + + - fireworks.jpeg is Copyright 2013 Steinar H. Gunderson, and + is licensed under the Creative Commons Attribution 3.0 license + (CC-BY-3.0). See https://creativecommons.org/licenses/by/3.0/ + for more information. + + - kppkn.gtb is taken from the Gaviota chess tablebase set, and + is licensed under the MIT License. See + https://sites.google.com/site/gaviotachessengine/Home/endgame-tablebases-1 + for more information. + + - paper-100k.pdf is an excerpt (bytes 92160 to 194560) from the paper + “Combinatorial Modeling of Chromatin Features Quantitatively Predicts DNA + Replication Timing in _Drosophila_” by Federico Comoglio and Renato Paro, + which is licensed under the CC-BY license. See + http://www.ploscompbiol.org/static/license for more ifnormation. + + - alice29.txt, asyoulik.txt, plrabn12.txt and lcet10.txt are from Project + Gutenberg. The first three have expired copyrights and are in the public + domain; the latter does not have expired copyright, but is still in the + public domain according to the license information + (http://www.gutenberg.org/ebooks/53). + +-------------------------------------------------------------------------------- + +3rdparty dependency gflags is statically linked in certain binary +distributions, like the python wheels. gflags has the following license: + +Copyright (c) 2006, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency glog is statically linked in certain binary +distributions, like the python wheels. glog has the following license: + +Copyright (c) 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +A function gettimeofday in utilities.cc is based on + +http://www.google.com/codesearch/p?hl=en#dR3YEbitojA/COPYING&q=GetSystemTimeAsFileTime%20license:bsd + +The license of this code is: + +Copyright (c) 2003-2008, Jouni Malinen and contributors +All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name(s) of the above-listed copyright holder(s) nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency re2 is statically linked in certain binary +distributions, like the python wheels. re2 has the following license: + +Copyright (c) 2009 The RE2 Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency c-ares is statically linked in certain binary +distributions, like the python wheels. c-ares has the following license: + +# c-ares license + +Copyright (c) 2007 - 2018, Daniel Stenberg with many contributors, see AUTHORS +file. + +Copyright 1998 by the Massachusetts Institute of Technology. + +Permission to use, copy, modify, and distribute this software and its +documentation for any purpose and without fee is hereby granted, provided that +the above copyright notice appear in all copies and that both that copyright +notice and this permission notice appear in supporting documentation, and that +the name of M.I.T. not be used in advertising or publicity pertaining to +distribution of the software without specific, written prior permission. +M.I.T. makes no representations about the suitability of this software for any +purpose. It is provided "as is" without express or implied warranty. + +-------------------------------------------------------------------------------- + +3rdparty dependency zlib is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. In the future +this will likely change to static linkage. zlib has the following license: + +zlib.h -- interface of the 'zlib' general purpose compression library + version 1.2.11, January 15th, 2017 + + Copyright (C) 1995-2017 Jean-loup Gailly and Mark Adler + + This software is provided 'as-is', without any express or implied + warranty. In no event will the authors be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Jean-loup Gailly Mark Adler + jloup@gzip.org madler@alumni.caltech.edu + +-------------------------------------------------------------------------------- + +3rdparty dependency openssl is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. openssl +preceding version 3 has the following license: + + LICENSE ISSUES + ============== + + The OpenSSL toolkit stays under a double license, i.e. both the conditions of + the OpenSSL License and the original SSLeay license apply to the toolkit. + See below for the actual license texts. + + OpenSSL License + --------------- + +/* ==================================================================== + * Copyright (c) 1998-2019 The OpenSSL Project. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * + * 3. All advertising materials mentioning features or use of this + * software must display the following acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit. (http://www.openssl.org/)" + * + * 4. The names "OpenSSL Toolkit" and "OpenSSL Project" must not be used to + * endorse or promote products derived from this software without + * prior written permission. For written permission, please contact + * openssl-core@openssl.org. + * + * 5. Products derived from this software may not be called "OpenSSL" + * nor may "OpenSSL" appear in their names without prior written + * permission of the OpenSSL Project. + * + * 6. Redistributions of any form whatsoever must retain the following + * acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit (http://www.openssl.org/)" + * + * THIS SOFTWARE IS PROVIDED BY THE OpenSSL PROJECT ``AS IS'' AND ANY + * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR + * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE OpenSSL PROJECT OR + * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT + * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + * OF THE POSSIBILITY OF SUCH DAMAGE. + * ==================================================================== + * + * This product includes cryptographic software written by Eric Young + * (eay@cryptsoft.com). This product includes software written by Tim + * Hudson (tjh@cryptsoft.com). + * + */ + + Original SSLeay License + ----------------------- + +/* Copyright (C) 1995-1998 Eric Young (eay@cryptsoft.com) + * All rights reserved. + * + * This package is an SSL implementation written + * by Eric Young (eay@cryptsoft.com). + * The implementation was written so as to conform with Netscapes SSL. + * + * This library is free for commercial and non-commercial use as long as + * the following conditions are aheared to. The following conditions + * apply to all code found in this distribution, be it the RC4, RSA, + * lhash, DES, etc., code; not just the SSL code. The SSL documentation + * included with this distribution is covered by the same copyright terms + * except that the holder is Tim Hudson (tjh@cryptsoft.com). + * + * Copyright remains Eric Young's, and as such any Copyright notices in + * the code are not to be removed. + * If this package is used in a product, Eric Young should be given attribution + * as the author of the parts of the library used. + * This can be in the form of a textual message at program startup or + * in documentation (online or textual) provided with the package. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. All advertising materials mentioning features or use of this software + * must display the following acknowledgement: + * "This product includes cryptographic software written by + * Eric Young (eay@cryptsoft.com)" + * The word 'cryptographic' can be left out if the rouines from the library + * being used are not cryptographic related :-). + * 4. If you include any Windows specific code (or a derivative thereof) from + * the apps directory (application code) you must include an acknowledgement: + * "This product includes software written by Tim Hudson (tjh@cryptsoft.com)" + * + * THIS SOFTWARE IS PROVIDED BY ERIC YOUNG ``AS IS'' AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS + * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + * SUCH DAMAGE. + * + * The licence and distribution terms for any publically available version or + * derivative of this code cannot be changed. i.e. this code cannot simply be + * copied and put under another distribution licence + * [including the GNU Public Licence.] + */ + +-------------------------------------------------------------------------------- + +This project includes code from the rtools-backports project. + +* ci/scripts/PKGBUILD and ci/scripts/r_windows_build.sh are based on code + from the rtools-backports project. + +Copyright: Copyright (c) 2013 - 2019, Алексей and Jeroen Ooms. +All rights reserved. +Homepage: https://github.com/r-windows/rtools-backports +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +Some code from pandas has been adapted for the pyarrow codebase. pandas is +available under the 3-clause BSD license, which follows: + +pandas license +============== + +Copyright (c) 2011-2012, Lambda Foundry, Inc. and PyData Development Team +All rights reserved. + +Copyright (c) 2008-2011 AQR Capital Management, LLC +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the copyright holder nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +Some bits from DyND, in particular aspects of the build system, have been +adapted from libdynd and dynd-python under the terms of the BSD 2-clause +license + +The BSD 2-Clause License + + Copyright (C) 2011-12, Dynamic NDArray Developers + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Dynamic NDArray Developers list: + + * Mark Wiebe + * Continuum Analytics + +-------------------------------------------------------------------------------- + +Some source code from Ibis (https://github.com/cloudera/ibis) has been adapted +for PyArrow. Ibis is released under the Apache License, Version 2.0. + +-------------------------------------------------------------------------------- + +This project includes code from the autobrew project. + +* r/tools/autobrew and dev/tasks/homebrew-formulae/autobrew/apache-arrow.rb + are based on code from the autobrew project. + +Copyright (c) 2019, Jeroen Ooms +License: MIT +Homepage: https://github.com/jeroen/autobrew + +-------------------------------------------------------------------------------- + +dev/tasks/homebrew-formulae/apache-arrow.rb has the following license: + +BSD 2-Clause License + +Copyright (c) 2009-present, Homebrew contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +---------------------------------------------------------------------- + +cpp/src/arrow/vendored/base64.cpp has the following license + +ZLIB License + +Copyright (C) 2004-2017 René Nyffenegger + +This source code is provided 'as-is', without any express or implied +warranty. In no event will the author be held liable for any damages arising +from the use of this software. + +Permission is granted to anyone to use this software for any purpose, including +commercial applications, and to alter it and redistribute it freely, subject to +the following restrictions: + +1. The origin of this source code must not be misrepresented; you must not + claim that you wrote the original source code. If you use this source code + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + +2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original source code. + +3. This notice may not be removed or altered from any source distribution. + +René Nyffenegger rene.nyffenegger@adp-gmbh.ch + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/optional.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. diff --git a/README.md b/README.md index f392d2317..8764adbd4 100644 --- a/README.md +++ b/README.md @@ -1,3 +1,7 @@ +##### \* LEGAL NOTICE: Your use of this software and any required dependent software (the "Software Package") is subject to the terms and conditions of the software license agreements for the Software Package, which may also include notices, disclaimers, or license terms for third party or open source software included in or with the Software Package, and your use indicates your acceptance of all such terms. Please refer to the "TPP.txt" or other similarly-named text file included with the Software Package for additional details. + +##### \* Optimized Analytics Package for Spark* Platform is under Apache 2.0 (https://www.apache.org/licenses/LICENSE-2.0). + # Spark Native SQL Engine A Native Engine for Spark SQL with vectorized SIMD optimizations @@ -10,7 +14,7 @@ You can find the all the Native SQL Engine documents on the [project web page](h ![Overview](./docs/image/nativesql_arch.png) -Spark SQL works very well with structured row-based data. It used WholeStageCodeGen to improve the performance by Java JIT code. However Java JIT is usually not working very well on utilizing latest SIMD instructions, especially under complicated queries. [Apache Arrow](https://arrow.apache.org/) provided CPU-cache friendly columnar in-memory layout, its SIMD optimized kernels and LLVM based SQL engine Gandiva are also very efficient. Native SQL Engine used these technoligies and brought better performance to Spark SQL. +Spark SQL works very well with structured row-based data. It used WholeStageCodeGen to improve the performance by Java JIT code. However Java JIT is usually not working very well on utilizing latest SIMD instructions, especially under complicated queries. [Apache Arrow](https://arrow.apache.org/) provided CPU-cache friendly columnar in-memory layout, its SIMD optimized kernels and LLVM based SQL engine Gandiva are also very efficient. Native SQL Engine used these technologies and brought better performance to Spark SQL. ## Key Features @@ -40,29 +44,42 @@ We implemented columnar shuffle to improve the shuffle performance. With the col Please check the operator supporting details [here](./docs/operators.md) -## Build the Plugin +## How to use OAP: Native SQL Engine + +There are three ways to use OAP: Native SQL Engine, +1. Use precompiled jars +2. Building by Conda Environment +3. Building by Yourself + +### Use precompiled jars + +Please go to [OAP's Maven Central Repository](https://repo1.maven.org/maven2/com/intel/oap/) to find Native SQL Engine jars. +For usage, you will require below two jar files: +1. spark-arrow-datasource-standard--jar-with-dependencies.jar is located in com/intel/oap/spark-arrow-datasource-standard// +2. spark-columnar-core--jar-with-dependencies.jar is located in com/intel/oap/spark-columnar-core// +Please notice the files are fat jars shipped with our custom Arrow library and pre-compiled from our server(using GCC 9.3.0 and LLVM 7.0.1), which means you will require to pre-install GCC 9.3.0 and LLVM 7.0.1 in your system for normal usage. ### Building by Conda If you already have a working Hadoop Spark Cluster, we provide a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. Once finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find built `spark-columnar-core--jar-with-dependencies.jar` under `$HOME/miniconda2/envs/oapenv/oap_jars`. -Then you can just skip below steps and jump to Getting Started [Get Started](#get-started). +Then you can just skip below steps and jump to [Get Started](#get-started). ### Building by yourself If you prefer to build from the source code on your hand, please follow below steps to set up your environment. -### Prerequisite +#### Prerequisite + There are some requirements before you build the project. Please check the document [Prerequisite](./docs/Prerequisite.md) and make sure you have already installed the software in your system. If you are running a SPARK Cluster, please make sure all the software are installed in every single node. -### Installation -Please check the document [Installation Guide](./docs/Installation.md) +#### Installation -### Configuration & Testing -Please check the document [Configuration Guide](./docs/Configuration.md) +Please check the document [Installation Guide](./docs/Installation.md) ## Get started + To enable OAP NativeSQL Engine, the previous built jar `spark-columnar-core--jar-with-dependencies.jar` should be added to Spark configuration. We also recommend to use `spark-arrow-datasource-standard--jar-with-dependencies.jar`. We will demonstrate an example by using both jar files. SPARK related options are: @@ -75,6 +92,8 @@ SPARK related options are: For Spark Standalone Mode, please set the above value as relative path to the jar file. For Spark Yarn Cluster Mode, please set the above value as absolute path to the jar file. +More Configuration, please check the document [Configuration Guide](./docs/Configuration.md) + Example to run Spark Shell with ArrowDataSource jar file ``` ${SPARK_HOME}/bin/spark-shell \ diff --git a/TPP.txt b/TPP.txt new file mode 100644 index 000000000..af00b1636 --- /dev/null +++ b/TPP.txt @@ -0,0 +1,10639 @@ +OAP Third Party Programs File + + +This file contains the list of third party software ("third party programs") +contained in the Intel software and their required notices and/or license terms. +This third party software, even if included with the distribution of the Intel +software, may be governed by separate license terms, including without limitation, +third party license terms, other Intel software license terms, and open source +software license terms. These separate license terms govern your use of the third +party programs as set forth in the "third-party-programs.txt" or other similarly named text file. + + +Third party programs and their corresponding required notices and/or license terms are listed below. + +-------------------------------------------------------------------------------- +1. Libcuckoo + Copyright (C) 2013, Carnegie Mellon University and Intel Corporation + 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. + --------------------------- + The third-party libraries have their own licenses, as detailed in their source + files. + + oneCCL + Copyright Intel Corporation + + oneDAL + Copyright Intel Corporation + + XGBoost + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright (c) 2019 by Contributors + + 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. + +-------------------------------------------------------------------------------- +2. Apache Spark + + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +------------------------------------------------------------------------------------ +This project bundles some components that are also licensed under the Apache +License Version 2.0: + +commons-beanutils:commons-beanutils +org.apache.zookeeper:zookeeper +oro:oro +commons-configuration:commons-configuration +commons-digester:commons-digester +com.chuusai:shapeless_2.12 +com.googlecode.javaewah:JavaEWAH +com.twitter:chill-java +com.twitter:chill_2.12 +com.univocity:univocity-parsers +javax.jdo:jdo-api +joda-time:joda-time +net.sf.opencsv:opencsv +org.apache.derby:derby +org.ehcache:ehcache +org.objenesis:objenesis +org.roaringbitmap:RoaringBitmap +org.scalanlp:breeze-macros_2.12 +org.scalanlp:breeze_2.12 +org.typelevel:macro-compat_2.12 +org.yaml:snakeyaml +org.apache.xbean:xbean-asm7-shaded +com.squareup.okhttp3:logging-interceptor +com.squareup.okhttp3:okhttp +com.squareup.okio:okio +org.apache.spark:spark-catalyst_2.12 +org.apache.spark:spark-kvstore_2.12 +org.apache.spark:spark-launcher_2.12 +org.apache.spark:spark-mllib-local_2.12 +org.apache.spark:spark-network-common_2.12 +org.apache.spark:spark-network-shuffle_2.12 +org.apache.spark:spark-sketch_2.12 +org.apache.spark:spark-tags_2.12 +org.apache.spark:spark-unsafe_2.12 +commons-httpclient:commons-httpclient +com.vlkan:flatbuffers +com.ning:compress-lzf +io.airlift:aircompressor +io.dropwizard.metrics:metrics-core +io.dropwizard.metrics:metrics-graphite +io.dropwizard.metrics:metrics-json +io.dropwizard.metrics:metrics-jvm +io.dropwizard.metrics:metrics-jmx +org.iq80.snappy:snappy +com.clearspring.analytics:stream +com.jamesmurty.utils:java-xmlbuilder +commons-codec:commons-codec +commons-collections:commons-collections +io.fabric8:kubernetes-client +io.fabric8:kubernetes-model +io.fabric8:kubernetes-model-common +io.netty:netty-all +net.hydromatic:eigenbase-properties +net.sf.supercsv:super-csv +org.apache.arrow:arrow-format +org.apache.arrow:arrow-memory +org.apache.arrow:arrow-vector +org.apache.commons:commons-configuration2 +org.apache.commons:commons-crypto +org.apache.commons:commons-lang3 +org.apache.hadoop:hadoop-annotations +org.apache.hadoop:hadoop-auth +org.apache.hadoop:hadoop-client +org.apache.hadoop:hadoop-common +org.apache.hadoop:hadoop-hdfs +org.apache.hadoop:hadoop-hdfs-client +org.apache.hadoop:hadoop-mapreduce-client-app +org.apache.hadoop:hadoop-mapreduce-client-common +org.apache.hadoop:hadoop-mapreduce-client-core +org.apache.hadoop:hadoop-mapreduce-client-jobclient +org.apache.hadoop:hadoop-mapreduce-client-shuffle +org.apache.hadoop:hadoop-yarn-api +org.apache.hadoop:hadoop-yarn-client +org.apache.hadoop:hadoop-yarn-common +org.apache.hadoop:hadoop-yarn-server-common +org.apache.hadoop:hadoop-yarn-server-web-proxy +org.apache.httpcomponents:httpclient +org.apache.httpcomponents:httpcore +org.apache.kerby:kerb-admin +org.apache.kerby:kerb-client +org.apache.kerby:kerb-common +org.apache.kerby:kerb-core +org.apache.kerby:kerb-crypto +org.apache.kerby:kerb-identity +org.apache.kerby:kerb-server +org.apache.kerby:kerb-simplekdc +org.apache.kerby:kerb-util +org.apache.kerby:kerby-asn1 +org.apache.kerby:kerby-config +org.apache.kerby:kerby-pkix +org.apache.kerby:kerby-util +org.apache.kerby:kerby-xdr +org.apache.kerby:token-provider +org.apache.orc:orc-core +org.apache.orc:orc-mapreduce +org.mortbay.jetty:jetty +org.mortbay.jetty:jetty-util +com.jolbox:bonecp +org.json4s:json4s-ast_2.12 +org.json4s:json4s-core_2.12 +org.json4s:json4s-jackson_2.12 +org.json4s:json4s-scalap_2.12 +com.carrotsearch:hppc +com.fasterxml.jackson.core:jackson-annotations +com.fasterxml.jackson.core:jackson-core +com.fasterxml.jackson.core:jackson-databind +com.fasterxml.jackson.dataformat:jackson-dataformat-yaml +com.fasterxml.jackson.jaxrs:jackson-jaxrs-base +com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider +com.fasterxml.jackson.module:jackson-module-jaxb-annotations +com.fasterxml.jackson.module:jackson-module-paranamer +com.fasterxml.jackson.module:jackson-module-scala_2.12 +com.fasterxml.woodstox:woodstox-core +com.github.mifmif:generex +com.github.stephenc.jcip:jcip-annotations +com.google.code.findbugs:jsr305 +com.google.code.gson:gson +com.google.flatbuffers:flatbuffers-java +com.google.guava:guava +com.google.inject:guice +com.google.inject.extensions:guice-servlet +com.nimbusds:nimbus-jose-jwt +com.twitter:parquet-hadoop-bundle +commons-cli:commons-cli +commons-daemon:commons-daemon +commons-dbcp:commons-dbcp +commons-io:commons-io +commons-lang:commons-lang +commons-logging:commons-logging +commons-net:commons-net +commons-pool:commons-pool +io.fabric8:zjsonpatch +javax.inject:javax.inject +javax.validation:validation-api +log4j:apache-log4j-extras +log4j:log4j +net.minidev:accessors-smart +net.minidev:json-smart +net.sf.jpam:jpam +org.apache.avro:avro +org.apache.avro:avro-ipc +org.apache.avro:avro-mapred +org.apache.commons:commons-compress +org.apache.commons:commons-math3 +org.apache.curator:curator-client +org.apache.curator:curator-framework +org.apache.curator:curator-recipes +org.apache.directory.api:api-asn1-api +org.apache.directory.api:api-util +org.apache.directory.server:apacheds-i18n +org.apache.directory.server:apacheds-kerberos-codec +org.apache.htrace:htrace-core +org.apache.ivy:ivy +org.apache.geronimo.specs:geronimo-jcache_1.0_spec +org.apache.mesos:mesos +org.apache.parquet:parquet-column +org.apache.parquet:parquet-common +org.apache.parquet:parquet-encoding +org.apache.parquet:parquet-format +org.apache.parquet:parquet-hadoop +org.apache.parquet:parquet-jackson +org.apache.thrift:libfb303 +org.apache.thrift:libthrift +org.codehaus.jackson:jackson-core-asl +org.codehaus.jackson:jackson-mapper-asl +org.datanucleus:datanucleus-api-jdo +org.datanucleus:datanucleus-core +org.datanucleus:datanucleus-rdbms +org.lz4:lz4-java +org.xerial.snappy:snappy-java +stax:stax-api +xerces:xercesImpl +org.codehaus.jackson:jackson-jaxrs +org.codehaus.jackson:jackson-xc +org.eclipse.jetty:jetty-client +org.eclipse.jetty:jetty-continuation +org.eclipse.jetty:jetty-http +org.eclipse.jetty:jetty-io +org.eclipse.jetty:jetty-jndi +org.eclipse.jetty:jetty-plus +org.eclipse.jetty:jetty-proxy +org.eclipse.jetty:jetty-security +org.eclipse.jetty:jetty-server +org.eclipse.jetty:jetty-servlet +org.eclipse.jetty:jetty-servlets +org.eclipse.jetty:jetty-util +org.eclipse.jetty:jetty-webapp +org.eclipse.jetty:jetty-xml +org.scala-lang.modules:scala-xml_2.12 +com.github.joshelser:dropwizard-metrics-hadoop-metrics2-reporter +com.zaxxer.HikariCP +org.apache.hive:hive-beeline +org.apache.hive:hive-cli +org.apache.hive:hive-common +org.apache.hive:hive-exec +org.apache.hive:hive-jdbc +org.apache.hive:hive-llap-common +org.apache.hive:hive-metastore +org.apache.hive:hive-serde +org.apache.hive:hive-service-rpc +org.apache.hive:hive-shims-0.23 +org.apache.hive:hive-shims +org.apache.hive:hive-common +org.apache.hive:hive-shims-scheduler +org.apache.hive:hive-storage-api +org.apache.hive:hive-vector-code-gen +org.datanucleus:javax.jdo +com.tdunning:json +org.apache.velocity:velocity +org.apache.yetus:audience-annotations + +core/src/main/java/org/apache/spark/util/collection/TimSort.java +core/src/main/resources/org/apache/spark/ui/static/bootstrap* +core/src/main/resources/org/apache/spark/ui/static/jsonFormatter* +core/src/main/resources/org/apache/spark/ui/static/vis* +docs/js/vendor/bootstrap.js + + +------------------------------------------------------------------------------------ +This product bundles various third-party components under other open source licenses. +This section summarizes those components and their licenses. See licenses-binary/ +for text of these licenses. + + +BSD 2-Clause +------------ + +com.github.luben:zstd-jni +dnsjava:dnsjava +javolution:javolution +com.esotericsoftware:kryo-shaded +com.esotericsoftware:minlog +com.esotericsoftware:reflectasm +com.google.protobuf:protobuf-java +org.codehaus.janino:commons-compiler +org.codehaus.janino:janino +org.codehaus.woodstox:stax2-api +jline:jline +org.jodd:jodd-core +com.github.wendykierp:JTransforms +pl.edu.icm:JLargeArrays + + +BSD 3-Clause +------------ + +dk.brics.automaton:automaton +org.antlr:antlr-runtime +org.antlr:ST4 +org.antlr:stringtemplate +org.antlr:antlr4-runtime +antlr:antlr +com.github.fommil.netlib:core +com.google.re2j:re2j +com.thoughtworks.paranamer:paranamer +org.scala-lang:scala-compiler +org.scala-lang:scala-library +org.scala-lang:scala-reflect +org.scala-lang.modules:scala-parser-combinators_2.12 +org.fusesource.leveldbjni:leveldbjni-all +net.sourceforge.f2j:arpack_combined_all +xmlenc:xmlenc +net.sf.py4j:py4j +org.jpmml:pmml-model +org.jpmml:pmml-schema +org.threeten:threeten-extra + +python/lib/py4j-*-src.zip +python/pyspark/cloudpickle.py +python/pyspark/join.py +core/src/main/resources/org/apache/spark/ui/static/d3.min.js + +The CSS style for the navigation sidebar of the documentation was originally +submitted by Óscar Nájera for the scikit-learn project. The scikit-learn project +is distributed under the 3-Clause BSD license. + + +MIT License +----------- + +com.microsoft.sqlserver:mssql-jdbc +org.typelevel:spire_2.12 +org.typelevel:spire-macros_2.12 +org.typelevel:spire-platform_2.12 +org.typelevel:spire-util_2.12 +org.typelevel:algebra_2.12:jar +org.typelevel:cats-kernel_2.12 +org.typelevel:machinist_2.12 +net.razorvine:pyrolite +org.slf4j:jcl-over-slf4j +org.slf4j:jul-to-slf4j +org.slf4j:slf4j-api +org.slf4j:slf4j-log4j12 +com.github.scopt:scopt_2.12 + +core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js +core/src/main/resources/org/apache/spark/ui/static/*dataTables* +core/src/main/resources/org/apache/spark/ui/static/graphlib-dot.min.js +core/src/main/resources/org/apache/spark/ui/static/jquery* +core/src/main/resources/org/apache/spark/ui/static/sorttable.js +docs/js/vendor/anchor.min.js +docs/js/vendor/jquery* +docs/js/vendor/modernizer* + + +Common Development and Distribution License (CDDL) 1.0 +------------------------------------------------------ + +javax.activation:activation http://www.oracle.com/technetwork/java/javase/tech/index-jsp-138795.html +javax.xml.stream:stax-api https://jcp.org/en/jsr/detail?id=173 +javax.transaction:javax.transaction-api + + +Common Development and Distribution License (CDDL) 1.1 +------------------------------------------------------ + +javax.el:javax.el-api https://javaee.github.io/uel-ri/ +javax.servlet:javax.servlet-api https://javaee.github.io/servlet-spec/ +javax.servlet.jsp:jsp-api +javax.transaction:jta http://www.oracle.com/technetwork/java/index.html +javax.xml.bind:jaxb-api https://github.com/javaee/jaxb-v2 +org.glassfish.hk2:hk2-api https://github.com/javaee/glassfish +org.glassfish.hk2:hk2-locator (same) +org.glassfish.hk2:hk2-utils +org.glassfish.hk2:osgi-resource-locator +org.glassfish.hk2.external:aopalliance-repackaged +org.glassfish.hk2.external:javax.inject +org.glassfish.jersey.bundles.repackaged:jersey-guava +org.glassfish.jersey.containers:jersey-container-servlet +org.glassfish.jersey.containers:jersey-container-servlet-core +org.glassfish.jersey.core:jersey-client +org.glassfish.jersey.core:jersey-common +org.glassfish.jersey.core:jersey-server +org.glassfish.jersey.media:jersey-media-jaxb + + +Eclipse Distribution License (EDL) 1.0 +-------------------------------------- + +org.glassfish.jaxb:jaxb-runtime +jakarta.activation:jakarta.activation-api +jakarta.xml.bind:jakarta.xml.bind-api +com.sun.istack:istack-commons-runtime + + +Eclipse Public License (EPL) 2.0 +-------------------------------- + +jakarta.annotation:jakarta-annotation-api https://projects.eclipse.org/projects/ee4j.ca +jakarta.ws.rs:jakarta.ws.rs-api https://github.com/eclipse-ee4j/jaxrs-api +org.glassfish.hk2.external:jakarta.inject + + +Public Domain +------------- + +aopalliance:aopalliance +net.iharder:base64 +org.tukaani:xz + + +Creative Commons CC0 1.0 Universal Public Domain Dedication +----------------------------------------------------------- +(see LICENSE-CC0.txt) + +data/mllib/images/kittens/29.5.a_b_EGDP022204.jpg +data/mllib/images/kittens/54893.jpg +data/mllib/images/kittens/DP153539.jpg +data/mllib/images/kittens/DP802813.jpg +data/mllib/images/multi-channel/chr30.4.184.jpg + + + Copyright 2014 and onwards The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Export Control Notice + --------------------- + + This distribution includes cryptographic software. The country in which you currently reside may have restrictions on the import, possession, use, and/or re-export to another country, of encryption software. + + BEFORE using any encryption software, please check your country's laws, regulations and policies concerning the import, possession, or use, and re-export of encryption software, to see if this is permitted. See for more information. + + The U.S. Government Department of Commerce, Bureau of Industry and Security (BIS), has classified this software as Export Commodity Control Number (ECCN) 5D002.C.1, which includes information security software using or performing cryptographic functions with asymmetric algorithms. The form and manner of this Apache Software Foundation distribution makes it eligible for export under the License Exception ENC Technology Software Unrestricted (TSU) exception (see the BIS Export Administration Regulations, Section 740.13) for both object code and source code. + + The following provides more details on the included cryptographic software: + + This software uses Apache Commons Crypto (https://commons.apache.org/proper/commons-crypto/) to support authentication, and encryption and decryption of data sent across the network between services. + + + // ------------------------------------------------------------------ + // NOTICE file corresponding to the section 4d of The Apache License, + // Version 2.0, in this case for + // ------------------------------------------------------------------ + + Hive Beeline + Copyright 2016 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Apache Avro + Copyright 2009-2014 The Apache Software Foundation + + This product currently only contains code developed by authors + of specific components, as identified by the source code files; + if such notes are missing files have been created by + Tatu Saloranta. + + For additional credits (generally to people who reported problems) + see CREDITS file. + + Apache Commons Compress + Copyright 2002-2012 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + Apache Avro Mapred API + Copyright 2009-2014 The Apache Software Foundation + + Apache Avro IPC + Copyright 2009-2014 The Apache Software Foundation + + Objenesis + Copyright 2006-2013 Joe Walnes, Henri Tremblay, Leonardo Mesquita + + Apache XBean :: ASM shaded (repackaged) + Copyright 2005-2019 The Apache Software Foundation + + -------------------------------------- + + This product includes software developed at + OW2 Consortium (http://asm.ow2.org/) + + This product includes software developed by The Apache Software + Foundation (http://www.apache.org/). + + The binary distribution of this product bundles binaries of + org.iq80.leveldb:leveldb-api (https://github.com/dain/leveldb), which has the + following notices: + * Copyright 2011 Dain Sundstrom + * Copyright 2011 FuseSource Corp. http://fusesource.com + + The binary distribution of this product bundles binaries of + org.fusesource.hawtjni:hawtjni-runtime (https://github.com/fusesource/hawtjni), + which has the following notices: + * This product includes software developed by FuseSource Corp. + http://fusesource.com + * This product includes software developed at + Progress Software Corporation and/or its subsidiaries or affiliates. + * This product includes software developed by IBM Corporation and others. + + The binary distribution of this product bundles binaries of + Gson 2.2.4, + which has the following notices: + + + The Netty Project + ================= + + Please visit the Netty web site for more information: + + * http://netty.io/ + + Copyright 2014 The Netty Project + + The Netty Project licenses this file to you under the Apache License, + version 2.0 (the "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at: + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + License for the specific language governing permissions and limitations + under the License. + + Also, please refer to each LICENSE..txt file, which is located in + the 'license' directory of the distribution file, for the license terms of the + components that this product depends on. + + ------------------------------------------------------------------------------- + This product contains the extensions to Java Collections Framework which has + been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + + This product contains a modified version of Robert Harder's Public Domain + Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + + This product contains a modified portion of 'Webbit', an event based + WebSocket and HTTP server, which can be obtained at: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + + This product contains a modified portion of 'SLF4J', a simple logging + facade for Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + + This product contains a modified portion of 'Apache Harmony', an open source + Java SE, which can be obtained at: + + * NOTICE: + * license/NOTICE.harmony.txt + * LICENSE: + * license/LICENSE.harmony.txt (Apache License 2.0) + * HOMEPAGE: + * http://archive.apache.org/dist/harmony/ + + This product contains a modified portion of 'jbzip2', a Java bzip2 compression + and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * license/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + + This product contains a modified portion of 'libdivsufsort', a C API library to construct + the suffix array and the Burrows-Wheeler transformed string for any input string of + a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * license/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://github.com/y-256/libdivsufsort + + This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, + which can be obtained at: + + * LICENSE: + * license/LICENSE.jctools.txt (ASL2 License) + * HOMEPAGE: + * https://github.com/JCTools/JCTools + + This product optionally depends on 'JZlib', a re-implementation of zlib in + pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + + This product optionally depends on 'Compress-LZF', a Java library for encoding and + decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: + + * LICENSE: + * license/LICENSE.compress-lzf.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/ning/compress + + This product optionally depends on 'lz4', a LZ4 Java compression + and decompression library written by Adrien Grand. It can be obtained at: + + * LICENSE: + * license/LICENSE.lz4.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jpountz/lz4-java + + This product optionally depends on 'lzma-java', a LZMA Java compression + and decompression library, which can be obtained at: + + * LICENSE: + * license/LICENSE.lzma-java.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jponge/lzma-java + + This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression + and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * license/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + + This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data + interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/protobuf + + This product optionally depends on 'Bouncy Castle Crypto APIs' to generate + a temporary self-signed X.509 certificate when the JVM does not provide the + equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + + This product optionally depends on 'Snappy', a compression library produced + by Google Inc, which can be obtained at: + + * LICENSE: + * license/LICENSE.snappy.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/snappy + + This product optionally depends on 'JBoss Marshalling', an alternative Java + serialization API, which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://www.jboss.org/jbossmarshalling + + This product optionally depends on 'Caliper', Google's micro- + benchmarking framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.caliper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/google/caliper + + This product optionally depends on 'Apache Commons Logging', a logging + framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + + This product optionally depends on 'Apache Log4J', a logging framework, which + can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + + This product optionally depends on 'Aalto XML', an ultra-high performance + non-blocking XML processor, which can be obtained at: + + * LICENSE: + * license/LICENSE.aalto-xml.txt (Apache License 2.0) + * HOMEPAGE: + * http://wiki.fasterxml.com/AaltoHome + + This product contains a modified version of 'HPACK', a Java implementation of + the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: + + * LICENSE: + * license/LICENSE.hpack.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/twitter/hpack + + This product contains a modified portion of 'Apache Commons Lang', a Java library + provides utilities for the java.lang API, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-lang.txt (Apache License 2.0) + * HOMEPAGE: + * https://commons.apache.org/proper/commons-lang/ + + + This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build. + + * LICENSE: + * license/LICENSE.mvn-wrapper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/takari/maven-wrapper + + + The binary distribution of this product bundles binaries of + Commons Codec 1.4, + which has the following notices: + * src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.javacontains test data from http://aspell.net/test/orig/batch0.tab.Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + =============================================================================== + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright:Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + The binary distribution of this product bundles binaries of + Commons Lang 2.6, + which has the following notices: + * This product includes software from the Spring Framework,under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + The binary distribution of this product bundles binaries of + Apache Log4j 1.2.17, + which has the following notices: + * ResolverUtil.java + Copyright 2005-2006 Tim Fennell + Dumbster SMTP test server + Copyright 2004 Jason Paul Kitchen + TypeUtil.java + Copyright 2002-2012 Ramnivas Laddad, Juergen Hoeller, Chris Beams + + The binary distribution of this product bundles binaries of + Jetty 6.1.26, + which has the following notices: + * ============================================================== + Jetty Web Container + Copyright 1995-2016 Mort Bay Consulting Pty Ltd. + ============================================================== + + The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd + unless otherwise noted. + + Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + + Jetty may be distributed under either license. + + ------ + Eclipse + + The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + + The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + ------ + Oracle + + The following artifacts are CDDL + GPLv2 with classpath exception. + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + + ------ + Oracle OpenJDK + + If ALPN is used to negotiate HTTP/2 connections, then the following + artifacts may be included in the distribution or downloaded when ALPN + module is selected. + + * java.sun.security.ssl + + These artifacts replace/modify OpenJDK classes. The modififications + are hosted at github and both modified and original are under GPL v2 with + classpath exceptions. + http://openjdk.java.net/legal/gplv2+ce.html + + ------ + OW2 + + The following artifacts are licensed by the OW2 Foundation according to the + terms of http://asm.ow2.org/license.html + + org.ow2.asm:asm-commons + org.ow2.asm:asm + + ------ + Apache + + The following artifacts are ASL2 licensed. + + org.apache.taglibs:taglibs-standard-spec + org.apache.taglibs:taglibs-standard-impl + + ------ + MortBay + + The following artifacts are ASL2 licensed. Based on selected classes from + following Apache Tomcat jars, all ASL2 licensed. + + org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + + org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + ------ + Mortbay + + The following artifacts are CDDL + GPLv2 with classpath exception. + + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + org.eclipse.jetty.toolchain:jetty-schemas + + ------ + Assorted + + The UnixCrypt.java code implements the one way cryptography used by + Unix systems for simple password protection. Copyright 1996 Aki Yoshida, + modified April 2001 by Iris Van den Broeke, Daniel Deville. + Permission to use, copy, modify and distribute UnixCrypt + for non-commercial or commercial purposes and without fee is + granted provided that the copyright notice appears in all copies./ + + The binary distribution of this product bundles binaries of + Snappy for Java 1.0.4.1, + which has the following notices: + * This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + + This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + + This library contains statically linked libstdc++. This inclusion is allowed by + "GCC RUntime Library Exception" + http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + + == Contributors == + * Tatu Saloranta + * Providing benchmark suite + * Alec Wysoker + * Performance and memory usage improvement + + The binary distribution of this product bundles binaries of + Xerces2 Java Parser 2.9.1, + which has the following notices: + * ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xerces Java distribution. == + ========================================================================= + + Apache Xerces Java + Copyright 1999-2007 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + + Apache Commons Collections + Copyright 2001-2015 The Apache Software Foundation + + Apache Commons Configuration + Copyright 2001-2008 The Apache Software Foundation + + Apache Jakarta Commons Digester + Copyright 2001-2006 The Apache Software Foundation + + Apache Commons BeanUtils + Copyright 2000-2008 The Apache Software Foundation + + ApacheDS Protocol Kerberos Codec + Copyright 2003-2013 The Apache Software Foundation + + ApacheDS I18n + Copyright 2003-2013 The Apache Software Foundation + + Apache Directory API ASN.1 API + Copyright 2003-2013 The Apache Software Foundation + + Apache Directory LDAP API Utilities + Copyright 2003-2013 The Apache Software Foundation + + Curator Client + Copyright 2011-2015 The Apache Software Foundation + + htrace-core + Copyright 2015 The Apache Software Foundation + + ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the Apache Xerces Java distribution. == + ========================================================================= + + Portions of this software were originally based on the following: + - software copyright (c) 1999, IBM Corporation., http://www.ibm.com. + - software copyright (c) 1999, Sun Microsystems., http://www.sun.com. + - voluntary contributions made by Paul Eng on behalf of the + Apache Software Foundation that were originally developed at iClick, Inc., + software copyright (c) 1999. + + # Jackson JSON processor + + Jackson is a high-performance, Free/Open Source JSON processing library. + It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has + been in development since 2007. + It is currently developed by a community of developers, as well as supported + commercially by FasterXML.com. + + ## Licensing + + Jackson core and extension components may licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + ## Credits + + A list of contributors may be found from CREDITS file, which is included + in some artifacts (usually source distributions); but is always available + from the source code management (SCM) system project uses. + + Apache HttpCore + Copyright 2005-2017 The Apache Software Foundation + + Curator Recipes + Copyright 2011-2015 The Apache Software Foundation + + Curator Framework + Copyright 2011-2015 The Apache Software Foundation + + Apache Commons Lang + Copyright 2001-2016 The Apache Software Foundation + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + Apache Commons Math + Copyright 2001-2015 The Apache Software Foundation + + This product includes software developed for Orekit by + CS Systèmes d'Information (http://www.c-s.fr/) + Copyright 2010-2012 CS Systèmes d'Information + + Apache log4j + Copyright 2007 The Apache Software Foundation + + # Compress LZF + + This library contains efficient implementation of LZF compression format, + as well as additional helper classes that build on JDK-provided gzip (deflat) + codec. + + Library is licensed under Apache License 2.0, as per accompanying LICENSE file. + + ## Credit + + Library has been written by Tatu Saloranta (tatu.saloranta@iki.fi). + It was started at Ning, inc., as an official Open Source process used by + platform backend, but after initial versions has been developed outside of + Ning by supporting community. + + Other contributors include: + + * Jon Hartlaub (first versions of streaming reader/writer; unit tests) + * Cedrik Lime: parallel LZF implementation + + Various community members have contributed bug reports, and suggested minor + fixes; these can be found from file "VERSION.txt" in SCM. + + Apache Commons Net + Copyright 2001-2012 The Apache Software Foundation + + + Jackson core and extension components may be licensed under different licenses. + To find the details that apply to this artifact see the accompanying LICENSE file. + For more information, including possible other licensing options, contact + FasterXML.com (http://fasterxml.com). + + Apache Ivy (TM) + Copyright 2007-2014 The Apache Software Foundation + + Portions of Ivy were originally developed at + Jayasoft SARL (http://www.jayasoft.fr/) + and are licensed to the Apache Software Foundation under the + "Software Grant License Agreement" + + SSH and SFTP support is provided by the JCraft JSch package, + which is open source software, available under + the terms of a BSD style license. + The original software and related information is available + at http://www.jcraft.com/jsch/. + + + ORC Core + Copyright 2013-2018 The Apache Software Foundation + + Apache Commons Lang + Copyright 2001-2011 The Apache Software Foundation + + ORC MapReduce + Copyright 2013-2018 The Apache Software Foundation + + Apache Parquet Format + Copyright 2017 The Apache Software Foundation + + Arrow Vectors + Copyright 2017 The Apache Software Foundation + + Arrow Format + Copyright 2017 The Apache Software Foundation + + Arrow Memory + Copyright 2017 The Apache Software Foundation + + Apache Commons CLI + Copyright 2001-2009 The Apache Software Foundation + + Apache Commons Daemon + Copyright 1999-2019 The Apache Software Foundation + + Google Guice - Extensions - Servlet + Copyright 2006-2011 Google, Inc. + + Apache Commons IO + Copyright 2002-2012 The Apache Software Foundation + + Google Guice - Core Library + Copyright 2006-2011 Google, Inc. + + mesos + Copyright 2017 The Apache Software Foundation + + Apache Parquet Hadoop Bundle (Incubating) + Copyright 2015 The Apache Software Foundation + + Hive Query Language + Copyright 2016 The Apache Software Foundation + + Apache Extras Companion for log4j 1.2. + Copyright 2007 The Apache Software Foundation + + Hive Metastore + Copyright 2016 The Apache Software Foundation + + Apache Commons Logging + Copyright 2003-2013 The Apache Software Foundation + + ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, == + == Version 2.0, in this case for the DataNucleus distribution. == + ========================================================================= + + =================================================================== + This product includes software developed by many individuals, + including the following: + =================================================================== + Erik Bengtson + Andy Jefferson + + =================================================================== + This product has included contributions from some individuals, + including the following: + =================================================================== + + =================================================================== + This product includes software developed by many individuals, + including the following: + =================================================================== + Andy Jefferson + Erik Bengtson + Joerg von Frantzius + Marco Schulze + + =================================================================== + This product has included contributions from some individuals, + including the following: + =================================================================== + Barry Haddow + Ralph Ullrich + David Ezzio + Brendan de Beer + David Eaves + Martin Taal + Tony Lai + Roland Szabo + Anton Troshin (Timesten) + + =================================================================== + This product also includes software developed by the TJDO project + (http://tjdo.sourceforge.net/). + =================================================================== + + =================================================================== + This product also includes software developed by the Apache Commons project + (http://commons.apache.org/). + =================================================================== + + Apache Commons Pool + Copyright 1999-2009 The Apache Software Foundation + + Apache Commons DBCP + Copyright 2001-2010 The Apache Software Foundation + + Apache Java Data Objects (JDO) + Copyright 2005-2006 The Apache Software Foundation + + Apache Jakarta HttpClient + Copyright 1999-2007 The Apache Software Foundation + + Apache HttpClient + Copyright 1999-2017 The Apache Software Foundation + + Apache Commons Codec + Copyright 2002-2014 The Apache Software Foundation + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + ============================================================================= + = NOTICE file corresponding to section 4d of the Apache License Version 2.0 = + ============================================================================= + This product includes software developed by + Joda.org (http://www.joda.org/). + + =================================================================== + This product has included contributions from some individuals, + including the following: + =================================================================== + Joerg von Frantzius + Thomas Marti + Barry Haddow + Marco Schulze + Ralph Ullrich + David Ezzio + Brendan de Beer + David Eaves + Martin Taal + Tony Lai + Roland Szabo + Marcus Mennemeier + Xuan Baldauf + Eric Sultan + + Apache Thrift + Copyright 2006-2010 The Apache Software Foundation. + + ========================================================================= + == NOTICE file corresponding to section 4(d) of the Apache License, + == Version 2.0, in this case for the Apache Derby distribution. + == + == DO NOT EDIT THIS FILE DIRECTLY. IT IS GENERATED + == BY THE buildnotice TARGET IN THE TOP LEVEL build.xml FILE. + == + ========================================================================= + + Apache Derby + Copyright 2004-2015 The Apache Software Foundation + + ========================================================================= + + Portions of Derby were originally developed by + International Business Machines Corporation and are + licensed to the Apache Software Foundation under the + "Software Grant and Corporate Contribution License Agreement", + informally known as the "Derby CLA". + The following copyright notice(s) were affixed to portions of the code + with which this file is now or was at one time distributed + and are placed here unaltered. + + (C) Copyright 1997,2004 International Business Machines Corporation. All rights reserved. + + (C) Copyright IBM Corp. 2003. + + The portion of the functionTests under 'nist' was originally + developed by the National Institute of Standards and Technology (NIST), + an agency of the United States Department of Commerce, and adapted by + International Business Machines Corporation in accordance with the NIST + Software Acknowledgment and Redistribution document at + http://www.itl.nist.gov/div897/ctg/sql_form.htm + + The JDBC apis for small devices and JDBC3 (under java/stubs/jsr169 and + java/stubs/jdbc3) were produced by trimming sources supplied by the + Apache Harmony project. In addition, the Harmony SerialBlob and + SerialClob implementations are used. The following notice covers the Harmony sources: + + Portions of Harmony were originally developed by + Intel Corporation and are licensed to the Apache Software + Foundation under the "Software Grant and Corporate Contribution + License Agreement", informally known as the "Intel Harmony CLA". + + The Derby build relies on source files supplied by the Apache Felix + project. The following notice covers the Felix files: + + Apache Felix Main + Copyright 2008 The Apache Software Foundation + + I. Included Software + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + Licensed under the Apache License 2.0. + + This product includes software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + This product includes software from http://kxml.sourceforge.net. + Copyright (c) 2002,2003, Stefan Haustein, Oberhausen, Rhld., Germany. + Licensed under BSD License. + + II. Used Software + + This product uses software developed at + The OSGi Alliance (http://www.osgi.org/). + Copyright (c) OSGi Alliance (2000, 2007). + Licensed under the Apache License 2.0. + + III. License Summary + - Apache License 2.0 + - BSD License + + The Derby build relies on jar files supplied by the Apache Lucene + project. The following notice covers the Lucene files: + + Apache Lucene + Copyright 2013 The Apache Software Foundation + + Includes software from other Apache Software Foundation projects, + including, but not limited to: + - Apache Ant + - Apache Jakarta Regexp + - Apache Commons + - Apache Xerces + + ICU4J, (under analysis/icu) is licensed under an MIT styles license + and Copyright (c) 1995-2008 International Business Machines Corporation and others + + Some data files (under analysis/icu/src/data) are derived from Unicode data such + as the Unicode Character Database. See http://unicode.org/copyright.html for more + details. + + Brics Automaton (under core/src/java/org/apache/lucene/util/automaton) is + BSD-licensed, created by Anders Møller. See http://www.brics.dk/automaton/ + + The levenshtein automata tables (under core/src/java/org/apache/lucene/util/automaton) were + automatically generated with the moman/finenight FSA library, created by + Jean-Philippe Barrette-LaPierre. This library is available under an MIT license, + see http://sites.google.com/site/rrettesite/moman and + http://bitbucket.org/jpbarrette/moman/overview/ + + The class org.apache.lucene.util.WeakIdentityMap was derived from + the Apache CXF project and is Apache License 2.0. + + The Google Code Prettify is Apache License 2.0. + See http://code.google.com/p/google-code-prettify/ + + JUnit (junit-4.10) is licensed under the Common Public License v. 1.0 + See http://junit.sourceforge.net/cpl-v10.html + + This product includes code (JaspellTernarySearchTrie) from Java Spelling Checkin + g Package (jaspell): http://jaspell.sourceforge.net/ + License: The BSD License (http://www.opensource.org/licenses/bsd-license.php) + + The snowball stemmers in + analysis/common/src/java/net/sf/snowball + were developed by Martin Porter and Richard Boulton. + The snowball stopword lists in + analysis/common/src/resources/org/apache/lucene/analysis/snowball + were developed by Martin Porter and Richard Boulton. + The full snowball package is available from + http://snowball.tartarus.org/ + + The KStem stemmer in + analysis/common/src/org/apache/lucene/analysis/en + was developed by Bob Krovetz and Sergio Guzman-Lara (CIIR-UMass Amherst) + under the BSD-license. + + The Arabic,Persian,Romanian,Bulgarian, and Hindi analyzers (common) come with a default + stopword list that is BSD-licensed created by Jacques Savoy. These files reside in: + analysis/common/src/resources/org/apache/lucene/analysis/ar/stopwords.txt, + analysis/common/src/resources/org/apache/lucene/analysis/fa/stopwords.txt, + analysis/common/src/resources/org/apache/lucene/analysis/ro/stopwords.txt, + analysis/common/src/resources/org/apache/lucene/analysis/bg/stopwords.txt, + analysis/common/src/resources/org/apache/lucene/analysis/hi/stopwords.txt + See http://members.unine.ch/jacques.savoy/clef/index.html. + + The German,Spanish,Finnish,French,Hungarian,Italian,Portuguese,Russian and Swedish light stemmers + (common) are based on BSD-licensed reference implementations created by Jacques Savoy and + Ljiljana Dolamic. These files reside in: + analysis/common/src/java/org/apache/lucene/analysis/de/GermanLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/de/GermanMinimalStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/es/SpanishLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/fi/FinnishLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/fr/FrenchMinimalStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/hu/HungarianLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/it/ItalianLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/pt/PortugueseLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/ru/RussianLightStemmer.java + analysis/common/src/java/org/apache/lucene/analysis/sv/SwedishLightStemmer.java + + The Stempel analyzer (stempel) includes BSD-licensed software developed + by the Egothor project http://egothor.sf.net/, created by Leo Galambos, Martin Kvapil, + and Edmond Nolan. + + The Polish analyzer (stempel) comes with a default + stopword list that is BSD-licensed created by the Carrot2 project. The file resides + in stempel/src/resources/org/apache/lucene/analysis/pl/stopwords.txt. + See http://project.carrot2.org/license.html. + + The SmartChineseAnalyzer source code (smartcn) was + provided by Xiaoping Gao and copyright 2009 by www.imdict.net. + + WordBreakTestUnicode_*.java (under modules/analysis/common/src/test/) + is derived from Unicode data such as the Unicode Character Database. + See http://unicode.org/copyright.html for more details. + + The Morfologik analyzer (morfologik) includes BSD-licensed software + developed by Dawid Weiss and Marcin Miłkowski (http://morfologik.blogspot.com/). + + Morfologik uses data from Polish ispell/myspell dictionary + (http://www.sjp.pl/slownik/en/) licenced on the terms of (inter alia) + LGPL and Creative Commons ShareAlike. + + Morfologic includes data from BSD-licensed dictionary of Polish (SGJP) + (http://sgjp.pl/morfeusz/) + + Servlet-api.jar and javax.servlet-*.jar are under the CDDL license, the original + source code for this can be found at http://www.eclipse.org/jetty/downloads.php + + =========================================================================== + Kuromoji Japanese Morphological Analyzer - Apache Lucene Integration + =========================================================================== + + This software includes a binary and/or source version of data from + + mecab-ipadic-2.7.0-20070801 + + which can be obtained from + + http://atilika.com/releases/mecab-ipadic/mecab-ipadic-2.7.0-20070801.tar.gz + + or + + http://jaist.dl.sourceforge.net/project/mecab/mecab-ipadic/2.7.0-20070801/mecab-ipadic-2.7.0-20070801.tar.gz + + =========================================================================== + mecab-ipadic-2.7.0-20070801 Notice + =========================================================================== + + Nara Institute of Science and Technology (NAIST), + the copyright holders, disclaims all warranties with regard to this + software, including all implied warranties of merchantability and + fitness, in no event shall NAIST be liable for + any special, indirect or consequential damages or any damages + whatsoever resulting from loss of use, data or profits, whether in an + action of contract, negligence or other tortuous action, arising out + of or in connection with the use or performance of this software. + + A large portion of the dictionary entries + originate from ICOT Free Software. The following conditions for ICOT + Free Software applies to the current dictionary as well. + + Each User may also freely distribute the Program, whether in its + original form or modified, to any third party or parties, PROVIDED + that the provisions of Section 3 ("NO WARRANTY") will ALWAYS appear + on, or be attached to, the Program, which is distributed substantially + in the same form as set out herein and that such intended + distribution, if actually made, will neither violate or otherwise + contravene any of the laws and regulations of the countries having + jurisdiction over the User or the intended distribution itself. + + NO WARRANTY + + The program was produced on an experimental basis in the course of the + research and development conducted during the project and is provided + to users as so produced on an experimental basis. Accordingly, the + program is provided without any warranty whatsoever, whether express, + implied, statutory or otherwise. The term "warranty" used herein + includes, but is not limited to, any warranty of the quality, + performance, merchantability and fitness for a particular purpose of + the program and the nonexistence of any infringement or violation of + any right of any third party. + + Each user of the program will agree and understand, and be deemed to + have agreed and understood, that there is no warranty whatsoever for + the program and, accordingly, the entire risk arising from or + otherwise connected with the program is assumed by the user. + + Therefore, neither ICOT, the copyright holder, or any other + organization that participated in or was otherwise related to the + development of the program and their respective officials, directors, + officers and other employees shall be held liable for any and all + damages, including, without limitation, general, special, incidental + and consequential damages, arising out of or otherwise in connection + with the use or inability to use the program or any product, material + or result produced or otherwise obtained by using the program, + regardless of whether they have been advised of, or otherwise had + knowledge of, the possibility of such damages at any time during the + project or thereafter. Each user will be deemed to have agreed to the + foregoing by his or her commencement of use of the program. The term + "use" as used herein includes, but is not limited to, the use, + modification, copying and distribution of the program and the + production of secondary products from the program. + + In the case where the program, whether in its original form or + modified, was distributed or delivered to or received by a user from + any person, organization or entity other than ICOT, unless it makes or + grants independently of ICOT any specific warranty to the user in + writing, such person, organization or entity, will also be exempted + from and not be held liable to the user for any such damages as noted + above as far as the program is concerned. + + The Derby build relies on a jar file supplied by the JSON Simple + project, hosted at https://code.google.com/p/json-simple/. + The JSON simple jar file is licensed under the Apache 2.0 License. + + Hive CLI + Copyright 2016 The Apache Software Foundation + + Hive JDBC + Copyright 2016 The Apache Software Foundation + + + Chill is a set of Scala extensions for Kryo. + Copyright 2012 Twitter, Inc. + + Third Party Dependencies: + + Kryo 2.17 + BSD 3-Clause License + http://code.google.com/p/kryo + + Commons-Codec 1.7 + Apache Public License 2.0 + http://hadoop.apache.org + + + + Breeze is distributed under an Apache License V2.0 (See LICENSE) + + =============================================================================== + + Proximal algorithms outlined in Proximal.scala (package breeze.optimize.proximal) + are based on https://github.com/cvxgrp/proximal (see LICENSE for details) and distributed with + Copyright (c) 2014 by Debasish Das (Verizon), all rights reserved. + + =============================================================================== + + QuadraticMinimizer class in package breeze.optimize.proximal is distributed with Copyright (c) + 2014, Debasish Das (Verizon), all rights reserved. + + =============================================================================== + + NonlinearMinimizer class in package breeze.optimize.proximal is distributed with Copyright (c) + 2015, Debasish Das (Verizon), all rights reserved. + + + stream-lib + Copyright 2016 AddThis + + This product includes software developed by AddThis. + + This product also includes code adapted from: + + Apache Solr (http://lucene.apache.org/solr/) + Copyright 2014 The Apache Software Foundation + + Apache Mahout (http://mahout.apache.org/) + Copyright 2014 The Apache Software Foundation + + scala-xml + Copyright (c) 2002-2019 EPFL + Copyright (c) 2011-2019 Lightbend, Inc. + + scala-xml includes software developed at + LAMP/EPFL (https://lamp.epfl.ch/) and + Lightbend, Inc. (https://www.lightbend.com/). + + Licensed under the Apache License, Version 2.0 (the "License"). + 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. + + dropwizard-metrics-hadoop-metrics2-reporter + Copyright 2016 Josh Elser + + Hive Beeline + Copyright 2019 The Apache Software Foundation + + Hive CLI + Copyright 2019 The Apache Software Foundation + + Hive Common + Copyright 2019 The Apache Software Foundation + + Hive JDBC + Copyright 2019 The Apache Software Foundation + + Hive Query Language + Copyright 2019 The Apache Software Foundation + + Hive Llap Common + Copyright 2019 The Apache Software Foundation + + Hive Metastore + Copyright 2019 The Apache Software Foundation + + Hive Serde + Copyright 2019 The Apache Software Foundation + + Hive Service RPC + Copyright 2019 The Apache Software Foundation + + Hive Shims + Copyright 2019 The Apache Software Foundation + + Hive Shims 0.23 + Copyright 2019 The Apache Software Foundation + + Hive Shims Common + Copyright 2019 The Apache Software Foundation + + Hive Shims Scheduler + Copyright 2019 The Apache Software Foundation + + Hive Storage API + Copyright 2018 The Apache Software Foundation + + Hive Vector-Code-Gen Utilities + Copyright 2019 The Apache Software Foundation + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + Copyright 2015-2015 DataNucleus + + 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. + + Android JSON library + Copyright (C) 2010 The Android Open Source Project + + This product includes software developed by + The Android Open Source Project + + Apache Velocity + + Copyright (C) 2000-2007 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Apache Yetus - Audience Annotations + Copyright 2015-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Ehcache V3 + Copyright 2014-2016 Terracotta, Inc. + + The product includes software from the Apache Commons Lang project, + under the Apache License 2.0 (see: org.ehcache.impl.internal.classes.commonslang) + + Apache Geronimo JCache Spec 1.0 + Copyright 2003-2014 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Admin + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Client + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Common + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb core + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Crypto + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Identity + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Server + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerb Simple Kdc + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby-kerb Util + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby ASN1 Project + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby Config + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby PKIX Project + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby Util + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Kerby XDR Project + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Token provider + Copyright 2014-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Metrics + Copyright 2010-2013 Coda Hale and Yammer, Inc. + + This product includes software developed by Coda Hale and Yammer, Inc. + + This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, + LongAdder), which was released with the following comments: + + Written by Doug Lea with assistance from members of JCP JSR-166 + Expert Group and released to the public domain, as explained at + http://creativecommons.org/publicdomain/zero/1.0/ + + +-------------------------------------------------------------------------------- +3. Apache Hadoop + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- +This project bundles some components that are also licensed under the Apache +License Version 2.0: + + +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/nvd3-1.8.5.* (css and js files) +hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java +hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/TimeoutFuture.java + +com.aliyun:aliyun-java-sdk-core:3.4.0 +com.aliyun:aliyun-java-sdk-ecs:4.2.0 +com.aliyun:aliyun-java-sdk-ram:3.0.0 +com.aliyun:aliyun-java-sdk-sts:3.0.0 +com.aliyun.oss:aliyun-sdk-oss:3.4.1 +com.amazonaws:aws-java-sdk-bundle:1.11.563 +com.cedarsoftware:java-util:1.9.0 +com.cedarsoftware:json-io:2.5.1 +com.fasterxml.jackson.core:jackson-annotations:2.9.9 +com.fasterxml.jackson.core:jackson-core:2.9.9 +com.fasterxml.jackson.core:jackson-databind:2.9.9.2 +com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:2.9.9 +com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:2.9.9 +com.fasterxml.jackson.module:jackson-module-jaxb-annotations:2.9.9 +com.fasterxml.uuid:java-uuid-generator:3.1.4 +com.fasterxml.woodstox:woodstox-core:5.0.3 +com.github.davidmoten:rxjava-extras:0.8.0.17 +com.github.stephenc.jcip:jcip-annotations:1.0-1 +com.google:guice:4.0 +com.google:guice-servlet:4.0 +com.google.api.grpc:proto-google-common-protos:1.0.0 +com.google.code.gson:2.2.4 +com.google.errorprone:error_prone_annotations:2.2.0 +com.google.j2objc:j2objc-annotations:1.1 +com.google.json-simple:json-simple:1.1.1 +com.google.guava:failureaccess:1.0 +com.google.guava:guava:20.0 +com.google.guava:guava:27.0-jre +com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava +com.microsoft.azure:azure-storage:7.0.0 +com.nimbusds:nimbus-jose-jwt:4.41.1 +com.squareup.okhttp:okhttp:2.7.5 +com.squareup.okio:okio:1.6.0 +com.zaxxer:HikariCP-java7:2.4.12 +commons-beanutils:commons-beanutils:1.9.3 +commons-cli:commons-cli:1.2 +commons-codec:commons-codec:1.11 +commons-collections:commons-collections:3.2.2 +commons-daemon:commons-daemon:1.0.13 +commons-io:commons-io:2.5 +commons-lang:commons-lang:2.6 +commons-logging:commons-logging:1.1.3 +commons-net:commons-net:3.6 +de.ruedigermoeller:fst:2.50 +io.grpc:grpc-api:1.26.0 +io.grpc:grpc-context:1.26.0 +io.grpc:grpc-core:1.26.0 +io.grpc:grpc-netty:1.26.0 +io.grpc:grpc-protobuf:1.26.0 +io.grpc:grpc-protobuf-lite:1.26.0 +io.grpc:grpc-stub:1.26.0 +io.netty:netty:3.10.6.Final +io.netty:netty-all:4.1.42.Final +io.netty:netty-buffer:4.1.27.Final +io.netty:netty-codec:4.1.27.Final +io.netty:netty-codec-http:4.1.27.Final +io.netty:netty-codec-http2:4.1.27.Final +io.netty:netty-codec-socks:4.1.27.Final +io.netty:netty-common:4.1.27.Final +io.netty:netty-handler:4.1.27.Final +io.netty:netty-handler-proxy:4.1.27.Final +io.netty:netty-resolver:4.1.27.Final +io.netty:netty-transport:4.1.27.Final +io.opencensus:opencensus-api:0.12.3 +io.opencensus:opencensus-contrib-grpc-metrics:0.12.3 +io.reactivex:rxjava:1.3.8 +io.reactivex:rxjava-string:1.1.1 +io.reactivex:rxnetty:0.4.20 +io.swagger:swagger-annotations:1.5.4 +javax.inject:javax.inject:1 +log4j:log4j:1.2.17 +net.java.dev.jna:jna:5.2.0 +net.minidev:accessors-smart:1.2 +net.minidev:json-smart:2.3 +org.apache.avro:avro:1.7.7 +org.apache.commons:commons-collections4:4.2 +org.apache.commons:commons-compress:1.19 +org.apache.commons:commons-configuration2:2.1.1 +org.apache.commons:commons-csv:1.0 +org.apache.commons:commons-digester:1.8.1 +org.apache.commons:commons-lang3:3.7 +org.apache.commons:commons-math3:3.1.1 +org.apache.commons:commons-text:1.4 +org.apache.commons:commons-validator:1.6 +org.apache.curator:curator-client:2.13.0 +org.apache.curator:curator-framework:2.13.0 +org.apache.curator:curator-recipes:2.13.0 +org.apache.geronimo.specs:geronimo-jcache_1.0_spec:1.0-alpha-1 +org.apache.hbase:hbase-annotations:1.4.8 +org.apache.hbase:hbase-client:1.4.8 +org.apache.hbase:hbase-common:1.4.8 +org.apache.hbase:hbase-protocol:1.4.8 +org.apache.htrace:htrace-core:3.1.0-incubating +org.apache.htrace:htrace-core4:4.1.0-incubating +org.apache.httpcomponents:httpclient:4.5.6 +org.apache.httpcomponents:httpcore:4.4.10 +org.apache.kafka:kafka-clients:2.4.0 +org.apache.kerby:kerb-admin:1.0.1 +org.apache.kerby:kerb-client:1.0.1 +org.apache.kerby:kerb-common:1.0.1 +org.apache.kerby:kerb-core:1.0.1 +org.apache.kerby:kerb-crypto:1.0.1 +org.apache.kerby:kerb-identity:1.0.1 +org.apache.kerby:kerb-server:1.0.1 +org.apache.kerby:kerb-simplekdc:1.0.1 +org.apache.kerby:kerb-util:1.0.1 +org.apache.kerby:kerby-asn1:1.0.1 +org.apache.kerby:kerby-config:1.0.1 +org.apache.kerby:kerby-pkix:1.0.1 +org.apache.kerby:kerby-util:1.0.1 +org.apache.kerby:kerby-xdr:1.0.1 +org.apache.kerby:token-provider:1.0.1 +org.apache.yetus:audience-annotations:0.5.0 +org.apache.zookeeper:zookeeper:3.4.13 +org.codehaus.jackson:jackson-core-asl:1.9.13 +org.codehaus.jackson:jackson-jaxrs:1.9.13 +org.codehaus.jackson:jackson-mapper-asl:1.9.13 +org.codehaus.jackson:jackson-xc:1.9.13 +org.codehaus.jettison:jettison:1.1 +org.eclipse.jetty:jetty-annotations:9.3.27.v20190418 +org.eclipse.jetty:jetty-http:9.3.27.v20190418 +org.eclipse.jetty:jetty-io:9.3.27.v20190418 +org.eclipse.jetty:jetty-jndi:9.3.27.v20190418 +org.eclipse.jetty:jetty-plus:9.3.27.v20190418 +org.eclipse.jetty:jetty-security:9.3.27.v20190418 +org.eclipse.jetty:jetty-server:9.3.27.v20190418 +org.eclipse.jetty:jetty-servlet:9.3.27.v20190418 +org.eclipse.jetty:jetty-util:9.3.27.v20190418 +org.eclipse.jetty:jetty-util-ajax:9.3.27.v20190418 +org.eclipse.jetty:jetty-webapp:9.3.27.v20190418 +org.eclipse.jetty:jetty-xml:9.3.27.v20190418 +org.eclipse.jetty.websocket:javax-websocket-client-impl:9.3.27.v20190418 +org.eclipse.jetty.websocket:javax-websocket-server-impl:9.3.27.v20190418 +org.ehcache:ehcache:3.3.1 +org.lz4:lz4-java:1.6.0 +org.objenesis:objenesis:2.6 +org.xerial.snappy:snappy-java:1.0.5 +org.yaml:snakeyaml:1.16: +org.wildfly.openssl:wildfly-openssl:1.0.7.Final + + +-------------------------------------------------------------------------------- +This product bundles various third-party components under other open source +licenses. This section summarizes those components and their licenses. +See licenses-binary/ for text of these licenses. + + +BSD 2-Clause +------------ + +hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/compress/lz4/{lz4.h,lz4.c,lz4hc.h,lz4hc.c} +hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/util/tree.h +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/native/container-executor/impl/compat/{fstatat|openat|unlinkat}.h + +com.github.luben:zstd-jni:1.4.3-1 +dnsjava:dnsjava:2.1.7 +org.codehaus.woodstox:stax2-api:3.1.4 + + +BSD 3-Clause +------------ + +hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/* +hadoop-common-project/hadoop-common/src/main/native/gtest/gtest-all.cc +hadoop-common-project/hadoop-common/src/main/native/gtest/include/gtest/gtest.h +hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/util/bulk_crc32_x86.c +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/d3.v3.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/d3-3.5.17.min.js +leveldb v1.13 + +com.google.protobuf:protobuf-java:2.5.0 +com.google.protobuf:protobuf-java:3.6.1 +com.google.re2j:re2j:1.1 +com.jcraft:jsch:0.1.54 +com.thoughtworks.paranamer:paranamer:2.3 +jakarta.activation:jakarta.activation-api:1.2.1 +org.fusesource.leveldbjni:leveldbjni-all:1.8 +org.jline:jline:3.9.0 +org.hamcrest:hamcrest-core:1.3 +org.ow2.asm:asm:5.0.4 +org.ow2.asm:asm-commons:6.0 +org.ow2.asm:asm-tree:6.0 + + +MIT License +----------- + +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/angular-1.6.4.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/angular-nvd3-1.0.9.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/angular-route-1.6.4.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/bootstrap-3.4.1 +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.css +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dataTables.bootstrap.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-full-2.0.0.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/dust-helpers-1.1.1.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery-3.4.1.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/jquery.dataTables.min.js +hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/static/moment.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/bootstrap.min.js +hadoop-tools/hadoop-sls/src/main/html/js/thirdparty/jquery.js +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap.min.css +hadoop-tools/hadoop-sls/src/main/html/css/bootstrap-responsive.min.css +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/dt-1.10.18/* +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jt/jquery.jstree.js +hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/resources/TERMINAL + +bootstrap v3.3.6 +broccoli-asset-rev v2.4.2 +broccoli-funnel v1.0.1 +datatables v1.10.8 +em-helpers v0.5.13 +em-table v0.1.6 +ember v2.2.0 +ember-array-contains-helper v1.0.2 +ember-bootstrap v0.5.1 +ember-cli v1.13.13 +ember-cli-app-version v1.0.0 +ember-cli-babel v5.1.6 +ember-cli-content-security-policy v0.4.0 +ember-cli-dependency-checker v1.2.0 +ember-cli-htmlbars v1.0.2 +ember-cli-htmlbars-inline-precompile v0.3.1 +ember-cli-ic-ajax v0.2.1 +ember-cli-inject-live-reload v1.4.0 +ember-cli-jquery-ui v0.0.20 +ember-cli-qunit v1.2.1 +ember-cli-release v0.2.8 +ember-cli-shims v0.0.6 +ember-cli-sri v1.2.1 +ember-cli-test-loader v0.2.1 +ember-cli-uglify v1.2.0 +ember-d3 v0.1.0 +ember-data v2.1.0 +ember-disable-proxy-controllers v1.0.1 +ember-export-application-global v1.0.5 +ember-load-initializers v0.1.7 +ember-qunit v0.4.16 +ember-qunit-notifications v0.1.0 +ember-resolver v2.0.3 +ember-spin-spinner v0.2.3 +ember-truth-helpers v1.2.0 +jquery v2.1.4 +jquery-ui v1.11.4 +loader.js v3.3.0 +momentjs v2.10.6 +qunit v1.19.0 +select2 v4.0.0 +snippet-ss v1.11.0 +spin.js v2.3.2 + +com.microsoft.azure:azure-cosmosdb:2.4.5 +com.microsoft.azure:azure-cosmosdb-commons:2.4.5 +com.microsoft.azure:azure-cosmosdb-direct:2.4.5 +com.microsoft.azure:azure-cosmosdb-gateway:2.4.5 +com.microsoft.azure:azure-data-lake-store-sdk:2.3.3 +com.microsoft.azure:azure-keyvault-core:1.0.0 +com.microsoft.sqlserver:mssql-jdbc:6.2.1.jre7 +org.bouncycastle:bcpkix-jdk15on:1.60 +org.bouncycastle:bcprov-jdk15on:1.60 +org.checkerframework:checker-qual:2.5.2 +org.codehaus.mojo:animal-sniffer-annotations:1.17 +org.jruby.jcodings:jcodings:1.0.13 +org.jruby.joni:joni:2.1.2 +org.slf4j:jul-to-slf4j:jar:1.7.25 +org.ojalgo:ojalgo:43.0:compile +org.slf4j:jul-to-slf4j:1.7.25 +org.slf4j:slf4j-api:1.7.25 +org.slf4j:slf4j-log4j12:1.7.25 + + +CDDL 1.1 + GPLv2 with classpath exception +----------------------------------------- + +com.sun.jersey:jersey-client:1.19 +com.sun.jersey:jersey-core:1.19 +com.sun.jersey:jersey-guice:1.19 +com.sun.jersey:jersey-json:1.19 +com.sun.jersey:jersey-server:1.19 +com.sun.jersey:jersey-servlet:1.19 +com.sun.xml.bind:jaxb-impl:2.2.3-1 +javax.annotation:javax.annotation-api:1.3.2 +javax.servlet:javax.servlet-api:3.1.0 +javax.servlet.jsp:jsp-api:2.1 +javax.websocket:javax.websocket-api:1.0 +javax.ws.rs:jsr311-api:1.1.1 +javax.xml.bind:jaxb-api:2.2.11 + + +Eclipse Public License 1.0 +-------------------------- + +junit:junit:4.12 + + +HSQL License +------------ + +org.hsqldb:hsqldb:2.3.4 + + +JDOM License +------------ + +org.jdom:jdom:1.1 + + +Public Domain +------------- + +aopalliance:aopalliance:1.0 + + + + +Copyright 2006 and onwards The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Export Control Notice + --------------------- + + This distribution includes cryptographic software. The country in + which you currently reside may have restrictions on the import, + possession, use, and/or re-export to another country, of + encryption software. BEFORE using any encryption software, please + check your country's laws, regulations and policies concerning the + import, possession, or use, and re-export of encryption software, to + see if this is permitted. See for more + information. + + The U.S. Government Department of Commerce, Bureau of Industry and + Security (BIS), has classified this software as Export Commodity + Control Number (ECCN) 5D002.C.1, which includes information security + software using or performing cryptographic functions with asymmetric + algorithms. The form and manner of this Apache Software Foundation + distribution makes it eligible for export under the License Exception + ENC Technology Software Unrestricted (TSU) exception (see the BIS + Export Administration Regulations, Section 740.13) for both object + code and source code. + + The following provides more details on the included cryptographic software: + + This software uses the SSL libraries from the Jetty project written + by mortbay.org. + Hadoop Yarn Server Web Proxy uses the BouncyCastle Java + cryptography APIs written by the Legion of the Bouncy Castle Inc. + + // ------------------------------------------------------------------ + // NOTICE file corresponding to the section 4d of The Apache License, + // Version 2.0, in this case for + // ------------------------------------------------------------------ + + + Apache Yetus + Copyright 2008-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + --- + Additional licenses for the Apache Yetus Source/Website: + --- + + + See LICENSE for terms. + + + + Apache Avro + Copyright 2010 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + C JSON parsing provided by Jansson and + written by Petri Lehtinen. The original software is + available from http://www.digip.org/jansson/. + + + AWS SDK for Java + Copyright 2010-2014 Amazon.com, Inc. or its affiliates. All Rights Reserved. + + This product includes software developed by + Amazon Technologies, Inc (http://www.amazon.com/). + + ********************** + THIRD PARTY COMPONENTS + ********************** + This software includes third party software subject to the following copyrights: + - XML parsing and utility functions from JetS3t - Copyright 2006-2009 James Murty. + - PKCS#1 PEM encoded private key parsing and utility functions from oauth.googlecode.com - Copyright 1998-2010 AOL Inc. + + The licenses for these third party components are included in LICENSE.txt + + + Apache Commons BeanUtils + Copyright 2000-2016 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons CLI + Copyright 2001-2009 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Codec + Copyright 2002-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java + contains test data from http://aspell.net/test/orig/batch0.tab. + Copyright (C) 2002 Kevin Atkinson (kevina@gnu.org) + + =============================================================================== + + The content of package org.apache.commons.codec.language.bm has been translated + from the original php source code available at http://stevemorse.org/phoneticinfo.htm + with permission from the original authors. + Original source copyright: + Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + + + Apache Commons Collections + Copyright 2001-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Compress + Copyright 2002-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (https://www.apache.org/). + + The files in the package org.apache.commons.compress.archivers.sevenz + were derived from the LZMA SDK, version 9.20 (C/ and CPP/7zip/), + which has been placed in the public domain: + + "LZMA SDK is placed in the public domain." (http://www.7-zip.org/sdk.html) + + + Apache Commons Configuration + Copyright 2001-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons CSV + Copyright 2005-2014 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + src/main/resources/contract.txt + This file was downloaded from http://www.ferc.gov/docs-filing/eqr/soft-tools/sample-csv/contract.txt and contains neither copyright notice nor license. + + src/main/resources/transaction.txt + This file was downloaded from http://www.ferc.gov/docs-filing/eqr/soft-tools/sample-csv/transaction.txt and contains neither copyright notice nor license. + + src/test/resources/CSVFileParser/bom.csv + src/test/resources/CSVFileParser/test.csv + src/test/resources/CSVFileParser/test_default.txt + src/test/resources/CSVFileParser/test_default_comment.txt + src/test/resources/CSVFileParser/test_rfc4180.txt + src/test/resources/CSVFileParser/test_rfc4180_trim.txt + src/test/resources/CSVFileParser/testCSV85.csv + src/test/resources/CSVFileParser/testCSV85_default.txt + src/test/resources/CSVFileParser/testCSV85_ignoreEmpty.txt + These files are used as test data and test result specifications. + + + Apache Commons Daemon + Copyright 1999-2013 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Digester + Copyright 2001-2008 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons IO + Copyright 2002-2016 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Lang + Copyright 2001-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + This product includes software from the Spring Framework, + under the Apache License 2.0 (see: StringUtils.containsWhitespace()) + + + Apache Commons Logging + Copyright 2003-2013 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Math + Copyright 2001-2012 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + =============================================================================== + + The BracketFinder (package org.apache.commons.math3.optimization.univariate) + and PowellOptimizer (package org.apache.commons.math3.optimization.general) + classes are based on the Python code in module "optimize.py" (version 0.5) + developed by Travis E. Oliphant for the SciPy library (http://www.scipy.org/) + Copyright © 2003-2009 SciPy Developers. + =============================================================================== + + The LinearConstraint, LinearObjectiveFunction, LinearOptimizer, + RelationShip, SimplexSolver and SimplexTableau classes in package + org.apache.commons.math3.optimization.linear include software developed by + Benjamin McCann (http://www.benmccann.com) and distributed with + the following copyright: Copyright 2009 Google Inc. + =============================================================================== + + This product includes software developed by the + University of Chicago, as Operator of Argonne National + Laboratory. + The LevenbergMarquardtOptimizer class in package + org.apache.commons.math3.optimization.general includes software + translated from the lmder, lmpar and qrsolv Fortran routines + from the Minpack package + Minpack Copyright Notice (1999) University of Chicago. All rights reserved + =============================================================================== + + The GraggBulirschStoerIntegrator class in package + org.apache.commons.math3.ode.nonstiff includes software translated + from the odex Fortran routine developed by E. Hairer and G. Wanner. + Original source copyright: + Copyright (c) 2004, Ernst Hairer + =============================================================================== + + The EigenDecompositionImpl class in package + org.apache.commons.math3.linear includes software translated + from some LAPACK Fortran routines. Original source copyright: + Copyright (c) 1992-2008 The University of Tennessee. All rights reserved. + =============================================================================== + + The MersenneTwister class in package org.apache.commons.math3.random + includes software translated from the 2002-01-26 version of + the Mersenne-Twister generator written in C by Makoto Matsumoto and Takuji + Nishimura. Original source copyright: + Copyright (C) 1997 - 2002, Makoto Matsumoto and Takuji Nishimura, + All rights reserved + =============================================================================== + + The LocalizedFormatsTest class in the unit tests is an adapted version of + the OrekitMessagesTest class from the orekit library distributed under the + terms of the Apache 2 licence. Original source copyright: + Copyright 2010 CS Systèmes d'Information + =============================================================================== + + The HermiteInterpolator class and its corresponding test have been imported from + the orekit library distributed under the terms of the Apache 2 licence. Original + source copyright: + Copyright 2010-2012 CS Systèmes d'Information + =============================================================================== + + The creation of the package "o.a.c.m.analysis.integration.gauss" was inspired + by an original code donated by Sébastien Brisard. + =============================================================================== + + + The complete text of licenses and disclaimers associated with the the original + sources enumerated above at the time of code translation are in the LICENSE.txt + file. + + + Apache Commons Net + Copyright 2001-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Text + Copyright 2014-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Commons Validator + Copyright 2001-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Curator + Copyright 2013-2014 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Ehcache V3 + Copyright 2014-2016 Terracotta, Inc. + + The product includes software from the Apache Commons Lang project, + under the Apache License 2.0 (see: org.ehcache.impl.internal.classes.commonslang) + + + Apache Geronimo + Copyright 2003-2018 The Apache Software Foundation + + This product includes software developed by + The Apache Software Foundation (http://www.apache.org/). + + + Copyright 2014 The gRPC Authors + + 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. + + ----------------------------------------------------------------------- + + This product contains a modified portion of 'OkHttp', an open source + HTTP & SPDY client for Android and Java applications, which can be obtained + at: + + * LICENSE: + * okhttp/third_party/okhttp/LICENSE (Apache License 2.0) + * HOMEPAGE: + * https://github.com/square/okhttp + * LOCATION_IN_GRPC: + * okhttp/third_party/okhttp + + This product contains a modified portion of 'Netty', an open source + networking library, which can be obtained at: + + * LICENSE: + * netty/third_party/netty/LICENSE.txt (Apache License 2.0) + * HOMEPAGE: + * https://netty.io + * LOCATION_IN_GRPC: + * netty/third_party/netty + + + Apache HBase + Copyright 2007-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + -- + This product incorporates portions of the 'Hadoop' project + + Copyright 2007-2009 The Apache Software Foundation + + Licensed under the Apache License v2.0 + -- + Our Orca logo we got here: http://www.vectorfree.com/jumping-orca + It is licensed Creative Commons Attribution 3.0. + See https://creativecommons.org/licenses/by/3.0/us/ + We changed the logo by stripping the colored background, inverting + it and then rotating it some. + + Later we found that vectorfree.com image is not properly licensed. + The original is owned by vectorportal.com. The original was + relicensed so we could use it as Creative Commons Attribution 3.0. + The license is bundled with the download available here: + http://www.vectorportal.com/subcategory/205/KILLER-WHALE-FREE-VECTOR.eps/ifile/9136/detailtest.asp + -- + This product includes portions of the Bootstrap project v3.0.0 + + Copyright 2013 Twitter, Inc. + + Licensed under the Apache License v2.0 + + This product uses the Glyphicons Halflings icon set. + + http://glyphicons.com/ + + Copyright Jan Kovařík + + Licensed under the Apache License v2.0 as a part of the Bootstrap project. + + -- + This product includes portions of the Guava project v14 and v21, specifically + 'hbase-common/src/main/java/org/apache/hadoop/hbase/io/LimitInputStream.java' + 'hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java' + + Copyright (C) 2007 The Guava Authors + + Licensed under the Apache License, Version 2.0 + + + Apache HTrace + Copyright 2016 The Apache Software Foundation + + This product includes software developed at The Apache Software + Foundation (http://www.apache.org/). + + In addition, this product includes software dependencies. See + the accompanying LICENSE.txt for a listing of dependencies + that are NOT Apache licensed (with pointers to their licensing) + + Apache HTrace includes an Apache Thrift connector to Zipkin. Zipkin + is a distributed tracing system that is Apache 2.0 Licensed. + Copyright 2012 Twitter, Inc. + + + Apache HttpComponents Client + Copyright 1999-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache HttpComponents Core + Copyright 2005-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + ============================================================== + Jetty Web Container + Copyright 1995-2017 Mort Bay Consulting Pty Ltd. + ============================================================== + + The Jetty Web Container is Copyright Mort Bay Consulting Pty Ltd + unless otherwise noted. + + Jetty is dual licensed under both + + * The Apache 2.0 License + http://www.apache.org/licenses/LICENSE-2.0.html + + and + + * The Eclipse Public 1.0 License + http://www.eclipse.org/legal/epl-v10.html + + Jetty may be distributed under either license. + + ------ + Eclipse + + The following artifacts are EPL. + * org.eclipse.jetty.orbit:org.eclipse.jdt.core + + The following artifacts are EPL and ASL2. + * org.eclipse.jetty.orbit:javax.security.auth.message + + + The following artifacts are EPL and CDDL 1.0. + * org.eclipse.jetty.orbit:javax.mail.glassfish + + + ------ + Oracle + + The following artifacts are CDDL + GPLv2 with classpath exception. + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + * javax.servlet:javax.servlet-api + * javax.annotation:javax.annotation-api + * javax.transaction:javax.transaction-api + * javax.websocket:javax.websocket-api + + ------ + Oracle OpenJDK + + If ALPN is used to negotiate HTTP/2 connections, then the following + artifacts may be included in the distribution or downloaded when ALPN + module is selected. + + * java.sun.security.ssl + + These artifacts replace/modify OpenJDK classes. The modififications + are hosted at github and both modified and original are under GPL v2 with + classpath exceptions. + http://openjdk.java.net/legal/gplv2+ce.html + + + ------ + OW2 + + The following artifacts are licensed by the OW2 Foundation according to the + terms of http://asm.ow2.org/license.html + + org.ow2.asm:asm-commons + org.ow2.asm:asm + + + ------ + Apache + + The following artifacts are ASL2 licensed. + + org.apache.taglibs:taglibs-standard-spec + org.apache.taglibs:taglibs-standard-impl + + + ------ + MortBay + + The following artifacts are ASL2 licensed. Based on selected classes from + following Apache Tomcat jars, all ASL2 licensed. + + org.mortbay.jasper:apache-jsp + org.apache.tomcat:tomcat-jasper + org.apache.tomcat:tomcat-juli + org.apache.tomcat:tomcat-jsp-api + org.apache.tomcat:tomcat-el-api + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-api + org.apache.tomcat:tomcat-util-scan + org.apache.tomcat:tomcat-util + + org.mortbay.jasper:apache-el + org.apache.tomcat:tomcat-jasper-el + org.apache.tomcat:tomcat-el-api + + + ------ + Mortbay + + The following artifacts are CDDL + GPLv2 with classpath exception. + + https://glassfish.dev.java.net/nonav/public/CDDL+GPL.html + + org.eclipse.jetty.toolchain:jetty-schemas + + ------ + Assorted + + The UnixCrypt.java code implements the one way cryptography used by + Unix systems for simple password protection. Copyright 1996 Aki Yoshida, + modified April 2001 by Iris Van den Broeke, Daniel Deville. + Permission to use, copy, modify and distribute UnixCrypt + for non-commercial or commercial purposes and without fee is + granted provided that the copyright notice appears in all copies. + + + Apache Kafka + Copyright 2012 The Apache Software Foundation. + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache Kerby + Copyright 2015-2017 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Apache log4j + Copyright 2010 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + Metrics + Copyright 2010-2013 Coda Hale and Yammer, Inc. + + This product includes software developed by Coda Hale and Yammer, Inc. + + This product includes code derived from the JSR-166 project (ThreadLocalRandom, Striped64, + LongAdder), which was released with the following comments: + + Written by Doug Lea with assistance from members of JCP JSR-166 + Expert Group and released to the public domain, as explained at + http://creativecommons.org/publicdomain/zero/1.0/ + + + + The Netty Project + ================= + + Please visit the Netty web site for more information: + + * http://netty.io/ + + Copyright 2014 The Netty Project + + The Netty Project licenses this file to you under the Apache License, + version 2.0 (the "License"); you may not use this file except in compliance + with the License. You may obtain a copy of the License at: + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + License for the specific language governing permissions and limitations + under the License. + + Also, please refer to each LICENSE..txt file, which is located in + the 'license' directory of the distribution file, for the license terms of the + components that this product depends on. + + ------------------------------------------------------------------------------- + This product contains the extensions to Java Collections Framework which has + been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + + This product contains a modified version of Robert Harder's Public Domain + Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + + This product contains a modified portion of 'Webbit', an event based + WebSocket and HTTP server, which can be obtained at: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + + This product contains a modified portion of 'SLF4J', a simple logging + facade for Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + + This product contains a modified portion of 'Apache Harmony', an open source + Java SE, which can be obtained at: + + * NOTICE: + * license/NOTICE.harmony.txt + * LICENSE: + * license/LICENSE.harmony.txt (Apache License 2.0) + * HOMEPAGE: + * http://archive.apache.org/dist/harmony/ + + This product contains a modified portion of 'jbzip2', a Java bzip2 compression + and decompression library written by Matthew J. Francis. It can be obtained at: + + * LICENSE: + * license/LICENSE.jbzip2.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jbzip2/ + + This product contains a modified portion of 'libdivsufsort', a C API library to construct + the suffix array and the Burrows-Wheeler transformed string for any input string of + a constant-size alphabet written by Yuta Mori. It can be obtained at: + + * LICENSE: + * license/LICENSE.libdivsufsort.txt (MIT License) + * HOMEPAGE: + * https://github.com/y-256/libdivsufsort + + This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, + which can be obtained at: + + * LICENSE: + * license/LICENSE.jctools.txt (ASL2 License) + * HOMEPAGE: + * https://github.com/JCTools/JCTools + + This product optionally depends on 'JZlib', a re-implementation of zlib in + pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + + This product optionally depends on 'Compress-LZF', a Java library for encoding and + decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: + + * LICENSE: + * license/LICENSE.compress-lzf.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/ning/compress + + This product optionally depends on 'lz4', a LZ4 Java compression + and decompression library written by Adrien Grand. It can be obtained at: + + * LICENSE: + * license/LICENSE.lz4.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jpountz/lz4-java + + This product optionally depends on 'lzma-java', a LZMA Java compression + and decompression library, which can be obtained at: + + * LICENSE: + * license/LICENSE.lzma-java.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/jponge/lzma-java + + This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression + and decompression library written by William Kinney. It can be obtained at: + + * LICENSE: + * license/LICENSE.jfastlz.txt (MIT License) + * HOMEPAGE: + * https://code.google.com/p/jfastlz/ + + This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data + interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/protobuf + + This product optionally depends on 'Bouncy Castle Crypto APIs' to generate + a temporary self-signed X.509 certificate when the JVM does not provide the + equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + + This product optionally depends on 'Snappy', a compression library produced + by Google Inc, which can be obtained at: + + * LICENSE: + * license/LICENSE.snappy.txt (New BSD License) + * HOMEPAGE: + * https://github.com/google/snappy + + This product optionally depends on 'JBoss Marshalling', an alternative Java + serialization API, which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-marshalling.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://www.jboss.org/jbossmarshalling + + This product optionally depends on 'Caliper', Google's micro- + benchmarking framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.caliper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/google/caliper + + This product optionally depends on 'Apache Commons Logging', a logging + framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + + This product optionally depends on 'Apache Log4J', a logging framework, which + can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + + This product optionally depends on 'Aalto XML', an ultra-high performance + non-blocking XML processor, which can be obtained at: + + * LICENSE: + * license/LICENSE.aalto-xml.txt (Apache License 2.0) + * HOMEPAGE: + * http://wiki.fasterxml.com/AaltoHome + + This product contains a modified version of 'HPACK', a Java implementation of + the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: + + * LICENSE: + * license/LICENSE.hpack.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/twitter/hpack + + This product contains a modified portion of 'Apache Commons Lang', a Java library + provides utilities for the java.lang API, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-lang.txt (Apache License 2.0) + * HOMEPAGE: + * https://commons.apache.org/proper/commons-lang/ + + + This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build. + + * LICENSE: + * license/LICENSE.mvn-wrapper.txt (Apache License 2.0) + * HOMEPAGE: + * https://github.com/takari/maven-wrapper + + + This product includes software developed by Google + Snappy: http://code.google.com/p/snappy/ (New BSD License) + + This product includes software developed by Apache + PureJavaCrc32C from apache-hadoop-common http://hadoop.apache.org/ + (Apache 2.0 license) + + This library containd statically linked libstdc++. This inclusion is allowed by + "GCC RUntime Library Exception" + http://gcc.gnu.org/onlinedocs/libstdc++/manual/license.html + + == Contributors == + * Tatu Saloranta + * Providing benchmark suite + * Alec Wysoker + * Performance and memory usage improvement + + + Apache ZooKeeper + Copyright 2009-2018 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- +4. Apache ORC + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability contains + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + +APACHE ORC SUBCOMPONENTS: + +The Apache ORC project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + +For protobuf: + + Copyright 2008 Google Inc. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + Code generated by the Protocol Buffer compiler is owned by the owner + of the input file used when generating it. This code is not + standalone and requires a support library to be linked with it. This + support library is itself covered by the above license. + +For the site: + + Parts of the site formatting includes software developed by Tom Preston-Werner + that are licensed under the MIT License (MIT): + + (c) Copyright [2008-2015] Tom Preston-Werner + + Permission is hereby granted, free of charge, to any person obtaining a copy + of this software and associated documentation files (the "Software"), to deal + in the Software without restriction, including without limitation the rights + to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + copies of the Software, and to permit persons to whom the Software is + furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in all + copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + SOFTWARE. + +For snappy: + + Copyright 2011, Google Inc. + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +For zlib: + + (C) 1995-2017 Jean-loup Gailly and Mark Adler + + This software is provided 'as-is', without any express or implied + warranty. In no event will the authors be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Jean-loup Gailly Mark Adler + jloup@gzip.org madler@alumni.caltech.edu + + If you use the zlib library in a product, we would appreciate *not* receiving + lengthy legal documents to sign. The sources are provided for free but without + warranty of any kind. The library has been entirely written by Jean-loup + Gailly and Mark Adler; it does not include third-party code. + + If you redistribute modified sources, we would appreciate that you include in + the file ChangeLog history information documenting your changes. Please read + the FAQ for more information on the distribution of modified source versions. + +For orc.threeten: + + /* + * Copyright (c) 2007-present, Stephen Colebourne & Michael Nascimento Santos + * + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * * Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * + * * Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * * Neither the name of JSR-310 nor the names of its contributors + * may be used to endorse or promote products derived from this software + * without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR + * CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR + * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING + * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + + Copyright 2013 and onwards The Apache Software Foundation. + + This product includes software developed by The Apache Software + Foundation (http://www.apache.org/). + + This product includes software developed by Hewlett-Packard: + (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability contains + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + + APACHE ORC SUBCOMPONENTS: + + The Apache ORC project contains subcomponents with separate copyright + notices and license terms. Your use of the source code for the these + subcomponents is subject to the terms and conditions of the following + licenses. + + For protobuf: + + Copyright 2008 Google Inc. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + Code generated by the Protocol Buffer compiler is owned by the owner + of the input file used when generating it. This code is not + standalone and requires a support library to be linked with it. This + support library is itself covered by the above license. + + For the site: + + Parts of the site formatting includes software developed by Tom Preston-Werner + that are licensed under the MIT License (MIT): + + (c) Copyright [2008-2015] Tom Preston-Werner + + Permission is hereby granted, free of charge, to any person obtaining a copy + of this software and associated documentation files (the "Software"), to deal + in the Software without restriction, including without limitation the rights + to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + copies of the Software, and to permit persons to whom the Software is + furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in all + copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + SOFTWARE. + + For snappy: + + Copyright 2011, Google Inc. + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + For zlib: + + (C) 1995-2017 Jean-loup Gailly and Mark Adler + + This software is provided 'as-is', without any express or implied + warranty. In no event will the authors be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Jean-loup Gailly Mark Adler + jloup@gzip.org madler@alumni.caltech.edu + + If you use the zlib library in a product, we would appreciate *not* receiving + lengthy legal documents to sign. The sources are provided for free but without + warranty of any kind. The library has been entirely written by Jean-loup + Gailly and Mark Adler; it does not include third-party code. + + If you redistribute modified sources, we would appreciate that you include in + the file ChangeLog history information documenting your changes. Please read + the FAQ for more information on the distribution of modified source versions. + + For orc.threeten: + + /* + * Copyright (c) 2007-present, Stephen Colebourne & Michael Nascimento Santos + * + * All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions are met: + * + * * Redistributions of source code must retain the above copyright notice, + * this list of conditions and the following disclaimer. + * + * * Redistributions in binary form must reproduce the above copyright notice, + * this list of conditions and the following disclaimer in the documentation + * and/or other materials provided with the distribution. + * + * * Neither the name of JSR-310 nor the names of its contributors + * may be used to endorse or promote products derived from this software + * without specific prior written permission. + * + * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR + * CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, + * EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, + * PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR + * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF + * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING + * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS + * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + */ + +-------------------------------------------------------------------------------- +5. Apache Parquet MR + + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + +-------------------------------------------------------------------------------- + +This product includes code from Apache Avro. + +Copyright: 2014 The Apache Software Foundation. +Home page: https://avro.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Daniel Lemire's JavaFastPFOR project. The +"Lemire" bit packing source code produced by parquet-generator is derived from +the JavaFastPFOR project. + +Copyright: 2013 Daniel Lemire +Home page: http://lemire.me/en/ +Project page: https://github.com/lemire/JavaFastPFOR +License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Spark. + +* dev/merge_parquet_pr.py is based on Spark's dev/merge_spark_pr.py + +Copyright: 2014 The Apache Software Foundation. +Home page: https://spark.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Twitter's ElephantBird project. + +* parquet-hadoop's UnmaterializableRecordCounter.java includes code from + ElephantBird's LzoRecordReader.java + +Copyright: 2012-2014 Twitter +Home page: https://github.com/twitter/elephant-bird +License: http://www.apache.org/licenses/LICENSE-2.0 + + + Copyright 2014 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + + + This product includes parquet-tools, initially developed at ARRIS, Inc. with + the following copyright notice: + + Copyright 2013 ARRIS, Inc. + + 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. + + -------------------------------------------------------------------------------- + + This product includes parquet-protobuf, initially developed by Lukas Nalezenc + with the following copyright notice: + + Copyright 2013 Lukas Nalezenec. + + 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. + + -------------------------------------------------------------------------------- + + This product includes code from Apache Avro, which includes the following in + its NOTICE file: + + Apache Avro + Copyright 2010-2015 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + -------------------------------------------------------------------------------- + + This project includes code from Kite, developed at Cloudera, Inc. with + the following copyright notice: + + | Copyright 2013 Cloudera Inc. + | + | 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. + + -------------------------------------------------------------------------------- + + This project includes code from Netflix, Inc. with the following copyright + notice: + + | Copyright 2016 Netflix, Inc. + | + | 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. + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + -------------------------------------------------------------------------------- + + This product includes code from Apache Avro. + + Copyright: 2014 The Apache Software Foundation. + Home page: https://avro.apache.org/ + License: http://www.apache.org/licenses/LICENSE-2.0 + + -------------------------------------------------------------------------------- + + This project includes code from Daniel Lemire's JavaFastPFOR project. The + "Lemire" bit packing source code produced by parquet-generator is derived from + the JavaFastPFOR project. + + Copyright: 2013 Daniel Lemire + Home page: http://lemire.me/en/ + Project page: https://github.com/lemire/JavaFastPFOR + License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + + -------------------------------------------------------------------------------- + + This product includes code from Apache Spark. + + * dev/merge_parquet_pr.py is based on Spark's dev/merge_spark_pr.py + + Copyright: 2014 The Apache Software Foundation. + Home page: https://spark.apache.org/ + License: http://www.apache.org/licenses/LICENSE-2.0 + + -------------------------------------------------------------------------------- + + This product includes code from Twitter's ElephantBird project. + + * parquet-hadoop's UnmaterializableRecordCounter.java includes code from + ElephantBird's LzoRecordReader.java + + Copyright: 2012-2014 Twitter + Home page: https://github.com/twitter/elephant-bird + License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- +6. Apache Arrow + +Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- + +src/plasma/fling.cc and src/plasma/fling.h: Apache 2.0 + +Copyright 2013 Sharvil Nanavati + +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. + +-------------------------------------------------------------------------------- + +src/plasma/thirdparty/ae: Modified / 3-Clause BSD + +Copyright (c) 2006-2010, Salvatore Sanfilippo +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of Redis nor the names of its contributors may be used + to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +src/plasma/thirdparty/dlmalloc.c: CC0 + +This is a version (aka dlmalloc) of malloc/free/realloc written by +Doug Lea and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ Send questions, +comments, complaints, performance data, etc to dl@cs.oswego.edu + +-------------------------------------------------------------------------------- + +src/plasma/common.cc (some portions) + +Copyright (c) Austin Appleby (aappleby (AT) gmail) + +Some portions of this file are derived from code in the MurmurHash project + +All code is released to the public domain. For business purposes, Murmurhash is +under the MIT license. + +https://sites.google.com/site/murmurhash/ + +-------------------------------------------------------------------------------- + +src/arrow/util (some portions): Apache 2.0, and 3-clause BSD + +Some portions of this module are derived from code in the Chromium project, +copyright (c) Google inc and (c) The Chromium Authors and licensed under the +Apache 2.0 License or the under the 3-clause BSD license: + + Copyright (c) 2013 The Chromium Authors. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from Daniel Lemire's FrameOfReference project. + +https://github.com/lemire/FrameOfReference/blob/6ccaf9e97160f9a3b299e23a8ef739e711ef0c71/src/bpacking.cpp + +Copyright: 2013 Daniel Lemire +Home page: http://lemire.me/en/ +Project page: https://github.com/lemire/FrameOfReference +License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the TensorFlow project + +Copyright 2015 The TensorFlow Authors. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the NumPy project. + +https://github.com/numpy/numpy/blob/e1f191c46f2eebd6cb892a4bfe14d9dd43a06c4e/numpy/core/src/multiarray/multiarraymodule.c#L2910 + +https://github.com/numpy/numpy/blob/68fd82271b9ea5a9e50d4e761061dfcca851382a/numpy/core/src/multiarray/datetime.c + +Copyright (c) 2005-2017, NumPy Developers. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the NumPy Developers nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the Boost project + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from the FlatBuffers project + +Copyright 2014 Google Inc. + +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. + +-------------------------------------------------------------------------------- + +This project includes code from the tslib project + +Copyright 2015 Microsoft Corporation. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the jemalloc project + +https://github.com/jemalloc/jemalloc + +Copyright (C) 2002-2017 Jason Evans . +All rights reserved. +Copyright (C) 2007-2012 Mozilla Foundation. All rights reserved. +Copyright (C) 2009-2017 Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: +1. Redistributions of source code must retain the above copyright notice(s), + this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice(s), + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER(S) ``AS IS'' AND ANY EXPRESS +OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO +EVENT SHALL THE COPYRIGHT HOLDER(S) BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE +OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF +ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------------------- + +This project includes code from the Go project, BSD 3-clause license + PATENTS +weak patent termination clause +(https://github.com/golang/go/blob/master/PATENTS). + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the hs2client + +https://github.com/cloudera/hs2client + +Copyright 2016 Cloudera Inc. + +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. + +-------------------------------------------------------------------------------- + +The script ci/scripts/util_wait_for_it.sh has the following license + +Copyright (c) 2016 Giles Hall + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The script r/configure has the following license (MIT) + +Copyright (c) 2017, Jeroen Ooms and Jim Hester + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +cpp/src/arrow/util/logging.cc, cpp/src/arrow/util/logging.h and +cpp/src/arrow/util/logging-test.cc are adapted from +Ray Project (https://github.com/ray-project/ray) (Apache 2.0). + +Copyright (c) 2016 Ray Project (https://github.com/ray-project/ray) + +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. + +-------------------------------------------------------------------------------- +The files cpp/src/arrow/vendored/datetime/date.h, cpp/src/arrow/vendored/datetime/tz.h, +cpp/src/arrow/vendored/datetime/tz_private.h, cpp/src/arrow/vendored/datetime/ios.h, +cpp/src/arrow/vendored/datetime/ios.mm, +cpp/src/arrow/vendored/datetime/tz.cpp are adapted from +Howard Hinnant's date library (https://github.com/HowardHinnant/date) +It is licensed under MIT license. + +The MIT License (MIT) +Copyright (c) 2015, 2016, 2017 Howard Hinnant +Copyright (c) 2016 Adrian Colomitchi +Copyright (c) 2017 Florian Dang +Copyright (c) 2017 Paul Thompson +Copyright (c) 2018 Tomasz Kamiński + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/util/utf8.h includes code adapted from the page + https://bjoern.hoehrmann.de/utf-8/decoder/dfa/ +with the following license (MIT) + +Copyright (c) 2008-2009 Bjoern Hoehrmann + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/string_view.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/variant.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/xxhash/ have the following license +(BSD 2-Clause License) + +xxHash Library +Copyright (c) 2012-2014, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +You can contact the author at : +- xxHash homepage: http://www.xxhash.com +- xxHash source repository : https://github.com/Cyan4973/xxHash + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/double-conversion/ have the following license +(BSD 3-Clause License) + +Copyright 2006-2011, the V8 project authors. All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/uriparser/ have the following license +(BSD 3-Clause License) + +uriparser - RFC 3986 URI parsing library + +Copyright (C) 2007, Weijia Song +Copyright (C) 2007, Sebastian Pipping +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + * Redistributions of source code must retain the above + copyright notice, this list of conditions and the following + disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials + provided with the distribution. + + * Neither the name of the nor the names of its + contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE +COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED +OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files under dev/tasks/conda-recipes have the following license + +BSD 3-clause license +Copyright (c) 2015-2018, conda-forge +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR +TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/utf8cpp/ have the following license + +Copyright 2006 Nemanja Trifunovic + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from Apache Kudu. + + * cpp/cmake_modules/CompilerInfo.cmake is based on Kudu's cmake_modules/CompilerInfo.cmake + +Copyright: 2016 The Apache Software Foundation. +Home page: https://kudu.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Impala (incubating), formerly +Impala. The Impala code and rights were donated to the ASF as part of the +Incubator process after the initial code imports into Apache Parquet. + +Copyright: 2012 Cloudera, Inc. +Copyright: 2016 The Apache Software Foundation. +Home page: http://impala.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Aurora. + +* dev/release/{release,changelog,release-candidate} are based on the scripts from + Apache Aurora + +Copyright: 2016 The Apache Software Foundation. +Home page: https://aurora.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the Google styleguide. + +* cpp/build-support/cpplint.py is based on the scripts from the Google styleguide. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/styleguide +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from Snappy. + +* cpp/cmake_modules/{SnappyCMakeLists.txt,SnappyConfig.h} are based on code + from Google's Snappy project. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/snappy +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from the manylinux project. + +* python/manylinux1/scripts/{build_python.sh,python-tag-abi-tag.py, + requirements.txt} are based on code from the manylinux project. + +Copyright: 2016 manylinux +Homepage: https://github.com/pypa/manylinux +License: The MIT License (MIT) + +-------------------------------------------------------------------------------- + +This project includes code from the cymove project: + +* python/pyarrow/includes/common.pxd includes code from the cymove project + +The MIT License (MIT) +Copyright (c) 2019 Omer Ozarslan + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE +OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The projects includes code from the Ursabot project under the dev/archery +directory. + +License: BSD 2-Clause + +Copyright 2019 RStudio, Inc. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project include code from CMake. + +* cpp/cmake_modules/FindGTest.cmake is based on code from CMake. + +Copyright: Copyright 2000-2019 Kitware, Inc. and Contributors +Homepage: https://gitlab.kitware.com/cmake/cmake +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project include code from mingw-w64. + +* cpp/src/arrow/util/cpu-info.cc has a polyfill for mingw-w64 < 5 + +Copyright (c) 2009 - 2013 by the mingw-w64 project +Homepage: https://mingw-w64.org +License: Zope Public License (ZPL) Version 2.1. + +--------------------------------------------------------------------------------- + +This project include code from Google's Asylo project. + +* cpp/src/arrow/result.h is based on status_or.h + +Copyright (c) Copyright 2017 Asylo authors +Homepage: https://asylo.dev/ +License: Apache 2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Google's protobuf project + +* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN + +Copyright 2008 Google Inc. All rights reserved. +Homepage: https://developers.google.com/protocol-buffers/ +License: + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +3rdparty dependency LLVM is statically linked in certain binary distributions. +Additionally some sections of source code have been derived from sources in LLVM +and have been clearly labeled as such. LLVM has the following license: + +============================================================================== +The LLVM Project is under the Apache License v2.0 with LLVM Exceptions: +============================================================================== + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + +---- LLVM Exceptions to the Apache 2.0 License ---- + +As an exception, if, as a result of your compiling your source code, portions +of this Software are embedded into an Object form of such source code, you +may redistribute such embedded portions in such Object form without complying +with the conditions of Sections 4(a), 4(b) and 4(d) of the License. + +In addition, if you combine or link compiled forms of this Software with +software that is licensed under the GPLv2 ("Combined Software") and if a +court of competent jurisdiction determines that the patent provision (Section +3), the indemnity provision (Section 9) or other Section of the License +conflicts with the conditions of the GPLv2, you may retroactively and +prospectively choose to deem waived or otherwise exclude such Section(s) of +the License, but only in their entirety and only with respect to the Combined +Software. + +============================================================================== +Software from third parties included in the LLVM Project: +============================================================================== +The LLVM Project contains third party software which is under different license +terms. All such code will be identified clearly using at least one of two +mechanisms: +1) It will be in a separate directory tree with its own `LICENSE.txt` or + `LICENSE` file at the top containing the specific license and restrictions + which apply to that software, or +2) It will contain specific license and restriction terms at the top of every + file. + +-------------------------------------------------------------------------------- + +3rdparty dependency gRPC is statically linked in certain binary +distributions, like the python wheels. gRPC has the following license: + +Copyright 2014 gRPC authors. + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache Thrift is statically linked in certain binary +distributions, like the python wheels. Apache Thrift has the following license: + +Apache Thrift +Copyright (C) 2006 - 2019, The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache ORC is statically linked in certain binary +distributions, like the python wheels. Apache ORC has the following license: + +Apache ORC +Copyright 2013-2019 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by Hewlett-Packard: +(c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency zstd is statically linked in certain binary +distributions, like the python wheels. ZSTD has the following license: + +BSD License + +For Zstandard software + +Copyright (c) 2016-present, Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + * Neither the name Facebook nor the names of its contributors may be used to + endorse or promote products derived from this software without specific + prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency lz4 is statically linked in certain binary +distributions, like the python wheels. lz4 has the following license: + +LZ4 Library +Copyright (c) 2011-2016, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency Brotli is statically linked in certain binary +distributions, like the python wheels. Brotli has the following license: + +Copyright (c) 2009, 2010, 2013-2016 by the Brotli Authors. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency rapidjson is statically linked in certain binary +distributions, like the python wheels. rapidjson and its dependencies have the +following licenses: + +Tencent is pleased to support the open source community by making RapidJSON +available. + +Copyright (C) 2015 THL A29 Limited, a Tencent company, and Milo Yip. +All rights reserved. + +If you have downloaded a copy of the RapidJSON binary from Tencent, please note +that the RapidJSON binary is licensed under the MIT License. +If you have downloaded a copy of the RapidJSON source code from Tencent, please +note that RapidJSON source code is licensed under the MIT License, except for +the third-party components listed below which are subject to different license +terms. Your integration of RapidJSON into your own projects may require +compliance with the MIT License, as well as the other licenses applicable to +the third-party components included within RapidJSON. To avoid the problematic +JSON license in your own projects, it's sufficient to exclude the +bin/jsonchecker/ directory, as it's the only code under the JSON license. +A copy of the MIT License is included in this file. + +Other dependencies and licenses: + + Open Source Software Licensed Under the BSD License: + -------------------------------------------------------------------- + + The msinttypes r29 + Copyright (c) 2006-2013 Alexander Chemeris + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE REGENTS AND CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + + Open Source Software Licensed Under the JSON License: + -------------------------------------------------------------------- + + json.org + Copyright (c) 2002 JSON.org + All Rights Reserved. + + JSON_checker + Copyright (c) 2002 JSON.org + All Rights Reserved. + + + Terms of the JSON License: + --------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + + The Software shall be used for Good, not Evil. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + + + Terms of the MIT License: + -------------------------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included + in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency snappy is statically linked in certain binary +distributions, like the python wheels. snappy has the following license: + +Copyright 2011, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of Google Inc. nor the names of its contributors may be + used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +=== + +Some of the benchmark data in testdata/ is licensed differently: + + - fireworks.jpeg is Copyright 2013 Steinar H. Gunderson, and + is licensed under the Creative Commons Attribution 3.0 license + (CC-BY-3.0). See https://creativecommons.org/licenses/by/3.0/ + for more information. + + - kppkn.gtb is taken from the Gaviota chess tablebase set, and + is licensed under the MIT License. See + https://sites.google.com/site/gaviotachessengine/Home/endgame-tablebases-1 + for more information. + + - paper-100k.pdf is an excerpt (bytes 92160 to 194560) from the paper + “Combinatorial Modeling of Chromatin Features Quantitatively Predicts DNA + Replication Timing in _Drosophila_” by Federico Comoglio and Renato Paro, + which is licensed under the CC-BY license. See + http://www.ploscompbiol.org/static/license for more ifnormation. + + - alice29.txt, asyoulik.txt, plrabn12.txt and lcet10.txt are from Project + Gutenberg. The first three have expired copyrights and are in the public + domain; the latter does not have expired copyright, but is still in the + public domain according to the license information + (http://www.gutenberg.org/ebooks/53). + +-------------------------------------------------------------------------------- + +3rdparty dependency gflags is statically linked in certain binary +distributions, like the python wheels. gflags has the following license: + +Copyright (c) 2006, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency glog is statically linked in certain binary +distributions, like the python wheels. glog has the following license: + +Copyright (c) 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +A function gettimeofday in utilities.cc is based on + +http://www.google.com/codesearch/p?hl=en#dR3YEbitojA/COPYING&q=GetSystemTimeAsFileTime%20license:bsd + +The license of this code is: + +Copyright (c) 2003-2008, Jouni Malinen and contributors +All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name(s) of the above-listed copyright holder(s) nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency re2 is statically linked in certain binary +distributions, like the python wheels. re2 has the following license: + +Copyright (c) 2009 The RE2 Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency c-ares is statically linked in certain binary +distributions, like the python wheels. c-ares has the following license: + +# c-ares license + +Copyright (c) 2007 - 2018, Daniel Stenberg with many contributors, see AUTHORS +file. + +Copyright 1998 by the Massachusetts Institute of Technology. + +Permission to use, copy, modify, and distribute this software and its +documentation for any purpose and without fee is hereby granted, provided that +the above copyright notice appear in all copies and that both that copyright +notice and this permission notice appear in supporting documentation, and that +the name of M.I.T. not be used in advertising or publicity pertaining to +distribution of the software without specific, written prior permission. +M.I.T. makes no representations about the suitability of this software for any +purpose. It is provided "as is" without express or implied warranty. + +-------------------------------------------------------------------------------- + +3rdparty dependency zlib is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. In the future +this will likely change to static linkage. zlib has the following license: + +zlib.h -- interface of the 'zlib' general purpose compression library + version 1.2.11, January 15th, 2017 + + Copyright (C) 1995-2017 Jean-loup Gailly and Mark Adler + + This software is provided 'as-is', without any express or implied + warranty. In no event will the authors be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Jean-loup Gailly Mark Adler + jloup@gzip.org madler@alumni.caltech.edu + +-------------------------------------------------------------------------------- + +3rdparty dependency openssl is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. openssl +preceding version 3 has the following license: + + LICENSE ISSUES + ============== + + The OpenSSL toolkit stays under a double license, i.e. both the conditions of + the OpenSSL License and the original SSLeay license apply to the toolkit. + See below for the actual license texts. + + OpenSSL License + --------------- + +/* ==================================================================== + * Copyright (c) 1998-2019 The OpenSSL Project. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * + * 3. All advertising materials mentioning features or use of this + * software must display the following acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit. (http://www.openssl.org/)" + * + * 4. The names "OpenSSL Toolkit" and "OpenSSL Project" must not be used to + * endorse or promote products derived from this software without + * prior written permission. For written permission, please contact + * openssl-core@openssl.org. + * + * 5. Products derived from this software may not be called "OpenSSL" + * nor may "OpenSSL" appear in their names without prior written + * permission of the OpenSSL Project. + * + * 6. Redistributions of any form whatsoever must retain the following + * acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit (http://www.openssl.org/)" + * + * THIS SOFTWARE IS PROVIDED BY THE OpenSSL PROJECT ``AS IS'' AND ANY + * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR + * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE OpenSSL PROJECT OR + * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT + * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + * OF THE POSSIBILITY OF SUCH DAMAGE. + * ==================================================================== + * + * This product includes cryptographic software written by Eric Young + * (eay@cryptsoft.com). This product includes software written by Tim + * Hudson (tjh@cryptsoft.com). + * + */ + + Original SSLeay License + ----------------------- + +/* Copyright (C) 1995-1998 Eric Young (eay@cryptsoft.com) + * All rights reserved. + * + * This package is an SSL implementation written + * by Eric Young (eay@cryptsoft.com). + * The implementation was written so as to conform with Netscapes SSL. + * + * This library is free for commercial and non-commercial use as long as + * the following conditions are aheared to. The following conditions + * apply to all code found in this distribution, be it the RC4, RSA, + * lhash, DES, etc., code; not just the SSL code. The SSL documentation + * included with this distribution is covered by the same copyright terms + * except that the holder is Tim Hudson (tjh@cryptsoft.com). + * + * Copyright remains Eric Young's, and as such any Copyright notices in + * the code are not to be removed. + * If this package is used in a product, Eric Young should be given attribution + * as the author of the parts of the library used. + * This can be in the form of a textual message at program startup or + * in documentation (online or textual) provided with the package. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. All advertising materials mentioning features or use of this software + * must display the following acknowledgement: + * "This product includes cryptographic software written by + * Eric Young (eay@cryptsoft.com)" + * The word 'cryptographic' can be left out if the rouines from the library + * being used are not cryptographic related :-). + * 4. If you include any Windows specific code (or a derivative thereof) from + * the apps directory (application code) you must include an acknowledgement: + * "This product includes software written by Tim Hudson (tjh@cryptsoft.com)" + * + * THIS SOFTWARE IS PROVIDED BY ERIC YOUNG ``AS IS'' AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS + * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + * SUCH DAMAGE. + * + * The licence and distribution terms for any publically available version or + * derivative of this code cannot be changed. i.e. this code cannot simply be + * copied and put under another distribution licence + * [including the GNU Public Licence.] + */ + +-------------------------------------------------------------------------------- + +This project includes code from the rtools-backports project. + +* ci/scripts/PKGBUILD and ci/scripts/r_windows_build.sh are based on code + from the rtools-backports project. + +Copyright: Copyright (c) 2013 - 2019, Алексей and Jeroen Ooms. +All rights reserved. +Homepage: https://github.com/r-windows/rtools-backports +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +Some code from pandas has been adapted for the pyarrow codebase. pandas is +available under the 3-clause BSD license, which follows: + +pandas license +============== + +Copyright (c) 2011-2012, Lambda Foundry, Inc. and PyData Development Team +All rights reserved. + +Copyright (c) 2008-2011 AQR Capital Management, LLC +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the copyright holder nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +Some bits from DyND, in particular aspects of the build system, have been +adapted from libdynd and dynd-python under the terms of the BSD 2-clause +license + +The BSD 2-Clause License + + Copyright (C) 2011-12, Dynamic NDArray Developers + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Dynamic NDArray Developers list: + + * Mark Wiebe + * Continuum Analytics + +-------------------------------------------------------------------------------- + +Some source code from Ibis (https://github.com/cloudera/ibis) has been adapted +for PyArrow. Ibis is released under the Apache License, Version 2.0. + +-------------------------------------------------------------------------------- + +This project includes code from the autobrew project. + +* r/tools/autobrew and dev/tasks/homebrew-formulae/autobrew/apache-arrow.rb + are based on code from the autobrew project. + +Copyright (c) 2019, Jeroen Ooms +License: MIT +Homepage: https://github.com/jeroen/autobrew + +-------------------------------------------------------------------------------- + +dev/tasks/homebrew-formulae/apache-arrow.rb has the following license: + +BSD 2-Clause License + +Copyright (c) 2009-present, Homebrew contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +---------------------------------------------------------------------- + +cpp/src/arrow/vendored/base64.cpp has the following license + +ZLIB License + +Copyright (C) 2004-2017 René Nyffenegger + +This source code is provided 'as-is', without any express or implied +warranty. In no event will the author be held liable for any damages arising +from the use of this software. + +Permission is granted to anyone to use this software for any purpose, including +commercial applications, and to alter it and redistribute it freely, subject to +the following restrictions: + +1. The origin of this source code must not be misrepresented; you must not + claim that you wrote the original source code. If you use this source code + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + +2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original source code. + +3. This notice may not be removed or altered from any source distribution. + +René Nyffenegger rene.nyffenegger@adp-gmbh.ch + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/optional.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/musl/strptime.c has the following license + +Copyright © 2005-2020 Rich Felker, et al. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY +CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/cmake_modules/BuildUtils.cmake contains code from + +https://gist.github.com/cristianadam/ef920342939a89fae3e8a85ca9459b49 + +which is made available under the MIT license + +Copyright (c) 2019 Cristian Adam + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/portable-snippets/ contain code from + +https://github.com/nemequ/portable-snippets + +and have the following copyright notice: + +Each source file contains a preamble explaining the license situation +for that file, which takes priority over this file. With the +exception of some code pulled in from other repositories (such as +µnit, an MIT-licensed project which is used for testing), the code is +public domain, released using the CC0 1.0 Universal dedication (*). + +(*) https://creativecommons.org/publicdomain/zero/1.0/legalcode + + Copyright 2016-2019 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + This product includes software from the SFrame project (BSD, 3-clause). + * Copyright (C) 2015 Dato, Inc. + * Copyright (c) 2009 Carnegie Mellon University. + + This product includes software from the Feather project (Apache 2.0) + https://github.com/wesm/feather + + This product includes software from the DyND project (BSD 2-clause) + https://github.com/libdynd + + This product includes software from the LLVM project + * distributed under the University of Illinois Open Source + + This product includes software from the google-lint project + * Copyright (c) 2009 Google Inc. All rights reserved. + + This product includes software from the mman-win32 project + * Copyright https://code.google.com/p/mman-win32/ + * Licensed under the MIT License; + + This product includes software from the LevelDB project + * Copyright (c) 2011 The LevelDB Authors. All rights reserved. + * Use of this source code is governed by a BSD-style license that can be + * Moved from Kudu http://github.com/cloudera/kudu + + This product includes software from the CMake project + * Copyright 2001-2009 Kitware, Inc. + * Copyright 2012-2014 Continuum Analytics, Inc. + * All rights reserved. + + This product includes software from https://github.com/matthew-brett/multibuild (BSD 2-clause) + * Copyright (c) 2013-2016, Matt Terry and Matthew Brett; all rights reserved. + + This product includes software from the Ibis project (Apache 2.0) + * Copyright (c) 2015 Cloudera, Inc. + * https://github.com/cloudera/ibis + + This product includes software from Dremio (Apache 2.0) + * Copyright (C) 2017-2018 Dremio Corporation + * https://github.com/dremio/dremio-oss + + This product includes software from Google Guava (Apache 2.0) + * Copyright (C) 2007 The Guava Authors + * https://github.com/google/guava + + This product include software from CMake (BSD 3-Clause) + * CMake - Cross Platform Makefile Generator + * Copyright 2000-2019 Kitware, Inc. and Contributors + + The web site includes files generated by Jekyll. + + -------------------------------------------------------------------------------- + + This product includes code from Apache Kudu, which includes the following in + its NOTICE file: + + Apache Kudu + Copyright 2016 The Apache Software Foundation + + This product includes software developed at + The Apache Software Foundation (http://www.apache.org/). + + Portions of this software were developed at + Cloudera, Inc (http://www.cloudera.com/). + + -------------------------------------------------------------------------------- + + This product includes code from Apache ORC, which includes the following in + its NOTICE file: + + Apache ORC + Copyright 2013-2019 The Apache Software Foundation + + This product includes software developed by The Apache Software + Foundation (http://www.apache.org/). + + This product includes software developed by Hewlett-Packard: + (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- + +src/plasma/fling.cc and src/plasma/fling.h: Apache 2.0 + +Copyright 2013 Sharvil Nanavati + +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. + +-------------------------------------------------------------------------------- + +src/plasma/thirdparty/ae: Modified / 3-Clause BSD + +Copyright (c) 2006-2010, Salvatore Sanfilippo +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of Redis nor the names of its contributors may be used + to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +src/plasma/thirdparty/dlmalloc.c: CC0 + +This is a version (aka dlmalloc) of malloc/free/realloc written by +Doug Lea and released to the public domain, as explained at +http://creativecommons.org/publicdomain/zero/1.0/ Send questions, +comments, complaints, performance data, etc to dl@cs.oswego.edu + +-------------------------------------------------------------------------------- + +src/plasma/common.cc (some portions) + +Copyright (c) Austin Appleby (aappleby (AT) gmail) + +Some portions of this file are derived from code in the MurmurHash project + +All code is released to the public domain. For business purposes, Murmurhash is +under the MIT license. + +https://sites.google.com/site/murmurhash/ + +-------------------------------------------------------------------------------- + +src/arrow/util (some portions): Apache 2.0, and 3-clause BSD + +Some portions of this module are derived from code in the Chromium project, +copyright (c) Google inc and (c) The Chromium Authors and licensed under the +Apache 2.0 License or the under the 3-clause BSD license: + + Copyright (c) 2013 The Chromium Authors. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from Daniel Lemire's FrameOfReference project. + +https://github.com/lemire/FrameOfReference/blob/6ccaf9e97160f9a3b299e23a8ef739e711ef0c71/src/bpacking.cpp + +Copyright: 2013 Daniel Lemire +Home page: http://lemire.me/en/ +Project page: https://github.com/lemire/FrameOfReference +License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the TensorFlow project + +Copyright 2015 The TensorFlow Authors. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the NumPy project. + +https://github.com/numpy/numpy/blob/e1f191c46f2eebd6cb892a4bfe14d9dd43a06c4e/numpy/core/src/multiarray/multiarraymodule.c#L2910 + +https://github.com/numpy/numpy/blob/68fd82271b9ea5a9e50d4e761061dfcca851382a/numpy/core/src/multiarray/datetime.c + +Copyright (c) 2005-2017, NumPy Developers. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the NumPy Developers nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the Boost project + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from the FlatBuffers project + +Copyright 2014 Google Inc. + +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. + +-------------------------------------------------------------------------------- + +This project includes code from the tslib project + +Copyright 2015 Microsoft Corporation. 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. + +-------------------------------------------------------------------------------- + +This project includes code from the jemalloc project + +https://github.com/jemalloc/jemalloc + +Copyright (C) 2002-2017 Jason Evans . +All rights reserved. +Copyright (C) 2007-2012 Mozilla Foundation. All rights reserved. +Copyright (C) 2009-2017 Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: +1. Redistributions of source code must retain the above copyright notice(s), + this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright notice(s), + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER(S) ``AS IS'' AND ANY EXPRESS +OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO +EVENT SHALL THE COPYRIGHT HOLDER(S) BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE +OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF +ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +-------------------------------------------------------------------------------- + +This project includes code from the Go project, BSD 3-clause license + PATENTS +weak patent termination clause +(https://github.com/golang/go/blob/master/PATENTS). + +Copyright (c) 2009 The Go Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project includes code from the hs2client + +https://github.com/cloudera/hs2client + +Copyright 2016 Cloudera Inc. + +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. + +-------------------------------------------------------------------------------- + +The script ci/scripts/util_wait_for_it.sh has the following license + +Copyright (c) 2016 Giles Hall + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The script r/configure has the following license (MIT) + +Copyright (c) 2017, Jeroen Ooms and Jim Hester + +Permission is hereby granted, free of charge, to any person obtaining a copy of +this software and associated documentation files (the "Software"), to deal in +the Software without restriction, including without limitation the rights to +use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies +of the Software, and to permit persons to whom the Software is furnished to do +so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +cpp/src/arrow/util/logging.cc, cpp/src/arrow/util/logging.h and +cpp/src/arrow/util/logging-test.cc are adapted from +Ray Project (https://github.com/ray-project/ray) (Apache 2.0). + +Copyright (c) 2016 Ray Project (https://github.com/ray-project/ray) + +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. + +-------------------------------------------------------------------------------- +The files cpp/src/arrow/vendored/datetime/date.h, cpp/src/arrow/vendored/datetime/tz.h, +cpp/src/arrow/vendored/datetime/tz_private.h, cpp/src/arrow/vendored/datetime/ios.h, +cpp/src/arrow/vendored/datetime/ios.mm, +cpp/src/arrow/vendored/datetime/tz.cpp are adapted from +Howard Hinnant's date library (https://github.com/HowardHinnant/date) +It is licensed under MIT license. + +The MIT License (MIT) +Copyright (c) 2015, 2016, 2017 Howard Hinnant +Copyright (c) 2016 Adrian Colomitchi +Copyright (c) 2017 Florian Dang +Copyright (c) 2017 Paul Thompson +Copyright (c) 2018 Tomasz Kamiński + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/util/utf8.h includes code adapted from the page + https://bjoern.hoehrmann.de/utf-8/decoder/dfa/ +with the following license (MIT) + +Copyright (c) 2008-2009 Bjoern Hoehrmann + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/string_view.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/variant.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/xxhash/ have the following license +(BSD 2-Clause License) + +xxHash Library +Copyright (c) 2012-2014, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +You can contact the author at : +- xxHash homepage: http://www.xxhash.com +- xxHash source repository : https://github.com/Cyan4973/xxHash + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/double-conversion/ have the following license +(BSD 3-Clause License) + +Copyright 2006-2011, the V8 project authors. All rights reserved. +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/uriparser/ have the following license +(BSD 3-Clause License) + +uriparser - RFC 3986 URI parsing library + +Copyright (C) 2007, Weijia Song +Copyright (C) 2007, Sebastian Pipping +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + * Redistributions of source code must retain the above + copyright notice, this list of conditions and the following + disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials + provided with the distribution. + + * Neither the name of the nor the names of its + contributors may be used to endorse or promote products + derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE +COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) +HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, +STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED +OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files under dev/tasks/conda-recipes have the following license + +BSD 3-clause license +Copyright (c) 2015-2018, conda-forge +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +3. Neither the name of the copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR +TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/utf8cpp/ have the following license + +Copyright 2006 Nemanja Trifunovic + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This project includes code from Apache Kudu. + + * cpp/cmake_modules/CompilerInfo.cmake is based on Kudu's cmake_modules/CompilerInfo.cmake + +Copyright: 2016 The Apache Software Foundation. +Home page: https://kudu.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Impala (incubating), formerly +Impala. The Impala code and rights were donated to the ASF as part of the +Incubator process after the initial code imports into Apache Parquet. + +Copyright: 2012 Cloudera, Inc. +Copyright: 2016 The Apache Software Foundation. +Home page: http://impala.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Apache Aurora. + +* dev/release/{release,changelog,release-candidate} are based on the scripts from + Apache Aurora + +Copyright: 2016 The Apache Software Foundation. +Home page: https://aurora.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This project includes code from the Google styleguide. + +* cpp/build-support/cpplint.py is based on the scripts from the Google styleguide. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/styleguide +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from Snappy. + +* cpp/cmake_modules/{SnappyCMakeLists.txt,SnappyConfig.h} are based on code + from Google's Snappy project. + +Copyright: 2009 Google Inc. All rights reserved. +Homepage: https://github.com/google/snappy +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project includes code from the manylinux project. + +* python/manylinux1/scripts/{build_python.sh,python-tag-abi-tag.py, + requirements.txt} are based on code from the manylinux project. + +Copyright: 2016 manylinux +Homepage: https://github.com/pypa/manylinux +License: The MIT License (MIT) + +-------------------------------------------------------------------------------- + +This project includes code from the cymove project: + +* python/pyarrow/includes/common.pxd includes code from the cymove project + +The MIT License (MIT) +Copyright (c) 2019 Omer Ozarslan + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, +DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR +OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE +OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The projects includes code from the Ursabot project under the dev/archery +directory. + +License: BSD 2-Clause + +Copyright 2019 RStudio, Inc. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This project include code from CMake. + +* cpp/cmake_modules/FindGTest.cmake is based on code from CMake. + +Copyright: Copyright 2000-2019 Kitware, Inc. and Contributors +Homepage: https://gitlab.kitware.com/cmake/cmake +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +This project include code from mingw-w64. + +* cpp/src/arrow/util/cpu-info.cc has a polyfill for mingw-w64 < 5 + +Copyright (c) 2009 - 2013 by the mingw-w64 project +Homepage: https://mingw-w64.org +License: Zope Public License (ZPL) Version 2.1. + +--------------------------------------------------------------------------------- + +This project include code from Google's Asylo project. + +* cpp/src/arrow/result.h is based on status_or.h + +Copyright (c) Copyright 2017 Asylo authors +Homepage: https://asylo.dev/ +License: Apache 2.0 + +-------------------------------------------------------------------------------- + +This project includes code from Google's protobuf project + +* cpp/src/arrow/result.h ARROW_ASSIGN_OR_RAISE is based off ASSIGN_OR_RETURN + +Copyright 2008 Google Inc. All rights reserved. +Homepage: https://developers.google.com/protocol-buffers/ +License: + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Code generated by the Protocol Buffer compiler is owned by the owner +of the input file used when generating it. This code is not +standalone and requires a support library to be linked with it. This +support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +3rdparty dependency LLVM is statically linked in certain binary distributions. +Additionally some sections of source code have been derived from sources in LLVM +and have been clearly labeled as such. LLVM has the following license: + +============================================================================== +The LLVM Project is under the Apache License v2.0 with LLVM Exceptions: +============================================================================== + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + +---- LLVM Exceptions to the Apache 2.0 License ---- + +As an exception, if, as a result of your compiling your source code, portions +of this Software are embedded into an Object form of such source code, you +may redistribute such embedded portions in such Object form without complying +with the conditions of Sections 4(a), 4(b) and 4(d) of the License. + +In addition, if you combine or link compiled forms of this Software with +software that is licensed under the GPLv2 ("Combined Software") and if a +court of competent jurisdiction determines that the patent provision (Section +3), the indemnity provision (Section 9) or other Section of the License +conflicts with the conditions of the GPLv2, you may retroactively and +prospectively choose to deem waived or otherwise exclude such Section(s) of +the License, but only in their entirety and only with respect to the Combined +Software. + +============================================================================== +Software from third parties included in the LLVM Project: +============================================================================== +The LLVM Project contains third party software which is under different license +terms. All such code will be identified clearly using at least one of two +mechanisms: +1) It will be in a separate directory tree with its own `LICENSE.txt` or + `LICENSE` file at the top containing the specific license and restrictions + which apply to that software, or +2) It will contain specific license and restriction terms at the top of every + file. + +-------------------------------------------------------------------------------- + +3rdparty dependency gRPC is statically linked in certain binary +distributions, like the python wheels. gRPC has the following license: + +Copyright 2014 gRPC authors. + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache Thrift is statically linked in certain binary +distributions, like the python wheels. Apache Thrift has the following license: + +Apache Thrift +Copyright (C) 2006 - 2019, The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency Apache ORC is statically linked in certain binary +distributions, like the python wheels. Apache ORC has the following license: + +Apache ORC +Copyright 2013-2019 The Apache Software Foundation + +This product includes software developed by The Apache Software +Foundation (http://www.apache.org/). + +This product includes software developed by Hewlett-Packard: +(c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +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. + +-------------------------------------------------------------------------------- + +3rdparty dependency zstd is statically linked in certain binary +distributions, like the python wheels. ZSTD has the following license: + +BSD License + +For Zstandard software + +Copyright (c) 2016-present, Facebook, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + + * Neither the name Facebook nor the names of its contributors may be used to + endorse or promote products derived from this software without specific + prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency lz4 is statically linked in certain binary +distributions, like the python wheels. lz4 has the following license: + +LZ4 Library +Copyright (c) 2011-2016, Yann Collet +All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency Brotli is statically linked in certain binary +distributions, like the python wheels. Brotli has the following license: + +Copyright (c) 2009, 2010, 2013-2016 by the Brotli Authors. + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency rapidjson is statically linked in certain binary +distributions, like the python wheels. rapidjson and its dependencies have the +following licenses: + +Tencent is pleased to support the open source community by making RapidJSON +available. + +Copyright (C) 2015 THL A29 Limited, a Tencent company, and Milo Yip. +All rights reserved. + +If you have downloaded a copy of the RapidJSON binary from Tencent, please note +that the RapidJSON binary is licensed under the MIT License. +If you have downloaded a copy of the RapidJSON source code from Tencent, please +note that RapidJSON source code is licensed under the MIT License, except for +the third-party components listed below which are subject to different license +terms. Your integration of RapidJSON into your own projects may require +compliance with the MIT License, as well as the other licenses applicable to +the third-party components included within RapidJSON. To avoid the problematic +JSON license in your own projects, it's sufficient to exclude the +bin/jsonchecker/ directory, as it's the only code under the JSON license. +A copy of the MIT License is included in this file. + +Other dependencies and licenses: + + Open Source Software Licensed Under the BSD License: + -------------------------------------------------------------------- + + The msinttypes r29 + Copyright (c) 2006-2013 Alexander Chemeris + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of copyright holder nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE REGENTS AND CONTRIBUTORS ``AS IS'' AND ANY + EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED + WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE + DISCLAIMED. IN NO EVENT SHALL THE REGENTS AND CONTRIBUTORS BE LIABLE FOR + ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR + SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER + CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH + DAMAGE. + + Open Source Software Licensed Under the JSON License: + -------------------------------------------------------------------- + + json.org + Copyright (c) 2002 JSON.org + All Rights Reserved. + + JSON_checker + Copyright (c) 2002 JSON.org + All Rights Reserved. + + + Terms of the JSON License: + --------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included in + all copies or substantial portions of the Software. + + The Software shall be used for Good, not Evil. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + + + Terms of the MIT License: + -------------------------------------------------------------------- + + Permission is hereby granted, free of charge, to any person obtaining a + copy of this software and associated documentation files (the "Software"), + to deal in the Software without restriction, including without limitation + the rights to use, copy, modify, merge, publish, distribute, sublicense, + and/or sell copies of the Software, and to permit persons to whom the + Software is furnished to do so, subject to the following conditions: + + The above copyright notice and this permission notice shall be included + in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING + FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +3rdparty dependency snappy is statically linked in certain binary +distributions, like the python wheels. snappy has the following license: + +Copyright 2011, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of Google Inc. nor the names of its contributors may be + used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +=== + +Some of the benchmark data in testdata/ is licensed differently: + + - fireworks.jpeg is Copyright 2013 Steinar H. Gunderson, and + is licensed under the Creative Commons Attribution 3.0 license + (CC-BY-3.0). See https://creativecommons.org/licenses/by/3.0/ + for more information. + + - kppkn.gtb is taken from the Gaviota chess tablebase set, and + is licensed under the MIT License. See + https://sites.google.com/site/gaviotachessengine/Home/endgame-tablebases-1 + for more information. + + - paper-100k.pdf is an excerpt (bytes 92160 to 194560) from the paper + “Combinatorial Modeling of Chromatin Features Quantitatively Predicts DNA + Replication Timing in _Drosophila_” by Federico Comoglio and Renato Paro, + which is licensed under the CC-BY license. See + http://www.ploscompbiol.org/static/license for more ifnormation. + + - alice29.txt, asyoulik.txt, plrabn12.txt and lcet10.txt are from Project + Gutenberg. The first three have expired copyrights and are in the public + domain; the latter does not have expired copyright, but is still in the + public domain according to the license information + (http://www.gutenberg.org/ebooks/53). + +-------------------------------------------------------------------------------- + +3rdparty dependency gflags is statically linked in certain binary +distributions, like the python wheels. gflags has the following license: + +Copyright (c) 2006, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency glog is statically linked in certain binary +distributions, like the python wheels. glog has the following license: + +Copyright (c) 2008, Google Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +A function gettimeofday in utilities.cc is based on + +http://www.google.com/codesearch/p?hl=en#dR3YEbitojA/COPYING&q=GetSystemTimeAsFileTime%20license:bsd + +The license of this code is: + +Copyright (c) 2003-2008, Jouni Malinen and contributors +All Rights Reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + +3. Neither the name(s) of the above-listed copyright holder(s) nor the + names of its contributors may be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency re2 is statically linked in certain binary +distributions, like the python wheels. re2 has the following license: + +Copyright (c) 2009 The RE2 Authors. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + * Neither the name of Google Inc. nor the names of its contributors + may be used to endorse or promote products derived from this + software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +3rdparty dependency c-ares is statically linked in certain binary +distributions, like the python wheels. c-ares has the following license: + +# c-ares license + +Copyright (c) 2007 - 2018, Daniel Stenberg with many contributors, see AUTHORS +file. + +Copyright 1998 by the Massachusetts Institute of Technology. + +Permission to use, copy, modify, and distribute this software and its +documentation for any purpose and without fee is hereby granted, provided that +the above copyright notice appear in all copies and that both that copyright +notice and this permission notice appear in supporting documentation, and that +the name of M.I.T. not be used in advertising or publicity pertaining to +distribution of the software without specific, written prior permission. +M.I.T. makes no representations about the suitability of this software for any +purpose. It is provided "as is" without express or implied warranty. + +-------------------------------------------------------------------------------- + +3rdparty dependency zlib is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. In the future +this will likely change to static linkage. zlib has the following license: + +zlib.h -- interface of the 'zlib' general purpose compression library + version 1.2.11, January 15th, 2017 + + Copyright (C) 1995-2017 Jean-loup Gailly and Mark Adler + + This software is provided 'as-is', without any express or implied + warranty. In no event will the authors be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Jean-loup Gailly Mark Adler + jloup@gzip.org madler@alumni.caltech.edu + +-------------------------------------------------------------------------------- + +3rdparty dependency openssl is redistributed as a dynamically linked shared +library in certain binary distributions, like the python wheels. openssl +preceding version 3 has the following license: + + LICENSE ISSUES + ============== + + The OpenSSL toolkit stays under a double license, i.e. both the conditions of + the OpenSSL License and the original SSLeay license apply to the toolkit. + See below for the actual license texts. + + OpenSSL License + --------------- + +/* ==================================================================== + * Copyright (c) 1998-2019 The OpenSSL Project. All rights reserved. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * + * 1. Redistributions of source code must retain the above copyright + * notice, this list of conditions and the following disclaimer. + * + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in + * the documentation and/or other materials provided with the + * distribution. + * + * 3. All advertising materials mentioning features or use of this + * software must display the following acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit. (http://www.openssl.org/)" + * + * 4. The names "OpenSSL Toolkit" and "OpenSSL Project" must not be used to + * endorse or promote products derived from this software without + * prior written permission. For written permission, please contact + * openssl-core@openssl.org. + * + * 5. Products derived from this software may not be called "OpenSSL" + * nor may "OpenSSL" appear in their names without prior written + * permission of the OpenSSL Project. + * + * 6. Redistributions of any form whatsoever must retain the following + * acknowledgment: + * "This product includes software developed by the OpenSSL Project + * for use in the OpenSSL Toolkit (http://www.openssl.org/)" + * + * THIS SOFTWARE IS PROVIDED BY THE OpenSSL PROJECT ``AS IS'' AND ANY + * EXPRESSED OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR + * PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE OpenSSL PROJECT OR + * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT + * NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; + * LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, + * STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) + * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED + * OF THE POSSIBILITY OF SUCH DAMAGE. + * ==================================================================== + * + * This product includes cryptographic software written by Eric Young + * (eay@cryptsoft.com). This product includes software written by Tim + * Hudson (tjh@cryptsoft.com). + * + */ + + Original SSLeay License + ----------------------- + +/* Copyright (C) 1995-1998 Eric Young (eay@cryptsoft.com) + * All rights reserved. + * + * This package is an SSL implementation written + * by Eric Young (eay@cryptsoft.com). + * The implementation was written so as to conform with Netscapes SSL. + * + * This library is free for commercial and non-commercial use as long as + * the following conditions are aheared to. The following conditions + * apply to all code found in this distribution, be it the RC4, RSA, + * lhash, DES, etc., code; not just the SSL code. The SSL documentation + * included with this distribution is covered by the same copyright terms + * except that the holder is Tim Hudson (tjh@cryptsoft.com). + * + * Copyright remains Eric Young's, and as such any Copyright notices in + * the code are not to be removed. + * If this package is used in a product, Eric Young should be given attribution + * as the author of the parts of the library used. + * This can be in the form of a textual message at program startup or + * in documentation (online or textual) provided with the package. + * + * Redistribution and use in source and binary forms, with or without + * modification, are permitted provided that the following conditions + * are met: + * 1. Redistributions of source code must retain the copyright + * notice, this list of conditions and the following disclaimer. + * 2. Redistributions in binary form must reproduce the above copyright + * notice, this list of conditions and the following disclaimer in the + * documentation and/or other materials provided with the distribution. + * 3. All advertising materials mentioning features or use of this software + * must display the following acknowledgement: + * "This product includes cryptographic software written by + * Eric Young (eay@cryptsoft.com)" + * The word 'cryptographic' can be left out if the rouines from the library + * being used are not cryptographic related :-). + * 4. If you include any Windows specific code (or a derivative thereof) from + * the apps directory (application code) you must include an acknowledgement: + * "This product includes software written by Tim Hudson (tjh@cryptsoft.com)" + * + * THIS SOFTWARE IS PROVIDED BY ERIC YOUNG ``AS IS'' AND + * ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE + * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE + * ARE DISCLAIMED. IN NO EVENT SHALL THE AUTHOR OR CONTRIBUTORS BE LIABLE + * FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL + * DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS + * OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) + * HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT + * LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY + * OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF + * SUCH DAMAGE. + * + * The licence and distribution terms for any publically available version or + * derivative of this code cannot be changed. i.e. this code cannot simply be + * copied and put under another distribution licence + * [including the GNU Public Licence.] + */ + +-------------------------------------------------------------------------------- + +This project includes code from the rtools-backports project. + +* ci/scripts/PKGBUILD and ci/scripts/r_windows_build.sh are based on code + from the rtools-backports project. + +Copyright: Copyright (c) 2013 - 2019, Алексей and Jeroen Ooms. +All rights reserved. +Homepage: https://github.com/r-windows/rtools-backports +License: 3-clause BSD + +-------------------------------------------------------------------------------- + +Some code from pandas has been adapted for the pyarrow codebase. pandas is +available under the 3-clause BSD license, which follows: + +pandas license +============== + +Copyright (c) 2011-2012, Lambda Foundry, Inc. and PyData Development Team +All rights reserved. + +Copyright (c) 2008-2011 AQR Capital Management, LLC +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + * Neither the name of the copyright holder nor the names of any + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDER AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +Some bits from DyND, in particular aspects of the build system, have been +adapted from libdynd and dynd-python under the terms of the BSD 2-clause +license + +The BSD 2-Clause License + + Copyright (C) 2011-12, Dynamic NDArray Developers + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +Dynamic NDArray Developers list: + + * Mark Wiebe + * Continuum Analytics + +-------------------------------------------------------------------------------- + +Some source code from Ibis (https://github.com/cloudera/ibis) has been adapted +for PyArrow. Ibis is released under the Apache License, Version 2.0. + +-------------------------------------------------------------------------------- + +This project includes code from the autobrew project. + +* r/tools/autobrew and dev/tasks/homebrew-formulae/autobrew/apache-arrow.rb + are based on code from the autobrew project. + +Copyright (c) 2019, Jeroen Ooms +License: MIT +Homepage: https://github.com/jeroen/autobrew + +-------------------------------------------------------------------------------- + +dev/tasks/homebrew-formulae/apache-arrow.rb has the following license: + +BSD 2-Clause License + +Copyright (c) 2009-present, Homebrew contributors +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +---------------------------------------------------------------------- + +cpp/src/arrow/vendored/base64.cpp has the following license + +ZLIB License + +Copyright (C) 2004-2017 René Nyffenegger + +This source code is provided 'as-is', without any express or implied +warranty. In no event will the author be held liable for any damages arising +from the use of this software. + +Permission is granted to anyone to use this software for any purpose, including +commercial applications, and to alter it and redistribute it freely, subject to +the following restrictions: + +1. The origin of this source code must not be misrepresented; you must not + claim that you wrote the original source code. If you use this source code + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + +2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original source code. + +3. This notice may not be removed or altered from any source distribution. + +René Nyffenegger rene.nyffenegger@adp-gmbh.ch + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/optional.hpp has the following license + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/src/arrow/vendored/musl/strptime.c has the following license + +Copyright © 2005-2020 Rich Felker, et al. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. +IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY +CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, +TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE +SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- + +The file cpp/cmake_modules/BuildUtils.cmake contains code from + +https://gist.github.com/cristianadam/ef920342939a89fae3e8a85ca9459b49 + +which is made available under the MIT license + +Copyright (c) 2019 Cristian Adam + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all +copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE +SOFTWARE. + +-------------------------------------------------------------------------------- + +The files in cpp/src/arrow/vendored/portable-snippets/ contain code from + +https://github.com/nemequ/portable-snippets + +and have the following copyright notice: + +Each source file contains a preamble explaining the license situation +for that file, which takes priority over this file. With the +exception of some code pulled in from other repositories (such as +µnit, an MIT-licensed project which is used for testing), the code is +public domain, released using the CC0 1.0 Universal dedication (*). + +(*) https://creativecommons.org/publicdomain/zero/1.0/legalcode + +-------------------------------------------------------------------------------- +7. HPNL + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- + +This product bundles various third-party components under other open source licenses. +This section summarizes those components and their licenses. See licenses/ +for text of these licenses. + + +Boost Software License 1.0 +-------------------------------------- +src/test/* +src/chunk/ChunkMgr.cc +include/HPNL/ChunkMgr.h + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization +obtaining a copy of the software and accompanying documentation covered by +this license (the "Software") to use, reproduce, display, distribute, +execute, and transmit the Software, and to prepare derivative works of the +Software, and to permit third-parties to whom the Software is furnished to +do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including +the above license grant, this restriction and the following disclaimer, +must be included in all copies of the Software, in whole or in part, and +all derivative works of the Software, unless such copies or derivative +works are solely in the form of machine-executable object code generated by +a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT +SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE +FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, +ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER +DEALINGS IN THE SOFTWARE. + +BSD license +-------------------------------------- +src/chunk/ChunkMgr.cc +src/core/* +demultiplexer/* +external_demultiplexer/* +service/* +external_service/* +test/CoreTest.cc +include/HPNL/ChunkMgr.h +include/HPNL/Client.h +include/HPNL/Connection.h +include/HPNL/Server.h +java/native/com_intel_hpnl_core_C* +java/native/com_intel_hpnl_core_E* +java/native/com_intel_hpnl_core_R* + +Copyright (c) 2015-2019 Intel Corporation. All rights reserved. +Copyright (c) 2015-2019 Cisco Systems, Inc. All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE +COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, +BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN +ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +Apache License 2.0 +-------------------------------------- +java/hpnl/src/test/java/com/intel/hpnl/* + +Copyright (c) Copyright 2017 Remko Popma +Homepage: https://github.com/remkop/picocli + +-------------------------------------------------------------------------------- +8. HiBench + +========================================================================== +The following license applies to software from the +Apache Software Foundation. +It also applies to software from the Uncommons Watchmaker and Math +projects, Google Guava software, and MongoDB.org driver software +-------------------------------------------------------------------------- + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + +-------------------------------------------------------------------------------- +9. catchorg/Catch2 + + Boost Software License - Version 1.0 - August 17th, 2003 + + Permission is hereby granted, free of charge, to any person or organization + obtaining a copy of the software and accompanying documentation covered by + this license (the "Software") to use, reproduce, display, distribute, + execute, and transmit the Software, and to prepare derivative works of the + Software, and to permit third-parties to whom the Software is furnished to + do so, all subject to the following: + + The copyright notices in the Software and this entire statement, including + the above license grant, this restriction and the following disclaimer, + must be included in all copies of the Software, in whole or in part, and + all derivative works of the Software, unless such copies or derivative + works are solely in the form of machine-executable object code generated by + a source language processor. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT + SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE + FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, + ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER + DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- +10. Libfabric + + This software is available to you under a choice of one of two +licenses. You may choose to be licensed under the terms of the the +BSD license or the GNU General Public License (GPL) Version +2, both included below. + +Copyright (c) 2015-2019 Intel Corporation. All rights reserved. +Copyright (c) 2015-2019 Cisco Systems, Inc. All rights reserved. + +================================================================== + + BSD license + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following + disclaimer in the documentation and/or other materials provided + with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS +FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE +COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, +BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT +LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN +ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE +POSSIBILITY OF SUCH DAMAGE. + +================================================================== + + GNU GENERAL PUBLIC LICENSE + Version 2, June 1991 + + Copyright (C) 1989, 1991 Free Software Foundation, Inc. + 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + Everyone is permitted to copy and distribute verbatim copies + of this license document, but changing it is not allowed. + + Preamble + + The licenses for most software are designed to take away your +freedom to share and change it. By contrast, the GNU General Public +License is intended to guarantee your freedom to share and change free +software--to make sure the software is free for all its users. This +General Public License applies to most of the Free Software +Foundation's software and to any other program whose authors commit to +using it. (Some other Free Software Foundation software is covered by +the GNU Library General Public License instead.) You can apply it to +your programs, too. + + When we speak of free software, we are referring to freedom, not +price. Our General Public Licenses are designed to make sure that you +have the freedom to distribute copies of free software (and charge for +this service if you wish), that you receive source code or can get it +if you want it, that you can change the software or use pieces of it +in new free programs; and that you know you can do these things. + + To protect your rights, we need to make restrictions that forbid +anyone to deny you these rights or to ask you to surrender the rights. +These restrictions translate to certain responsibilities for you if you +distribute copies of the software, or if you modify it. + + For example, if you distribute copies of such a program, whether +gratis or for a fee, you must give the recipients all the rights that +you have. You must make sure that they, too, receive or can get the +source code. And you must show them these terms so they know their +rights. + + We protect your rights with two steps: (1) copyright the software, and +(2) offer you this license which gives you legal permission to copy, +distribute and/or modify the software. + + Also, for each author's protection and ours, we want to make certain +that everyone understands that there is no warranty for this free +software. If the software is modified by someone else and passed on, we +want its recipients to know that what they have is not the original, so +that any problems introduced by others will not reflect on the original +authors' reputations. + + Finally, any free program is threatened constantly by software +patents. We wish to avoid the danger that redistributors of a free +program will individually obtain patent licenses, in effect making the +program proprietary. To prevent this, we have made it clear that any +patent must be licensed for everyone's free use or not licensed at all. + + The precise terms and conditions for copying, distribution and +modification follow. + + GNU GENERAL PUBLIC LICENSE + TERMS AND CONDITIONS FOR COPYING, DISTRIBUTION AND MODIFICATION + + 0. This License applies to any program or other work which contains +a notice placed by the copyright holder saying it may be distributed +under the terms of this General Public License. The "Program", below, +refers to any such program or work, and a "work based on the Program" +means either the Program or any derivative work under copyright law: +that is to say, a work containing the Program or a portion of it, +either verbatim or with modifications and/or translated into another +language. (Hereinafter, translation is included without limitation in +the term "modification".) Each licensee is addressed as "you". + +Activities other than copying, distribution and modification are not +covered by this License; they are outside its scope. The act of +running the Program is not restricted, and the output from the Program +is covered only if its contents constitute a work based on the +Program (independent of having been made by running the Program). +Whether that is true depends on what the Program does. + + 1. You may copy and distribute verbatim copies of the Program's +source code as you receive it, in any medium, provided that you +conspicuously and appropriately publish on each copy an appropriate +copyright notice and disclaimer of warranty; keep intact all the +notices that refer to this License and to the absence of any warranty; +and give any other recipients of the Program a copy of this License +along with the Program. + +You may charge a fee for the physical act of transferring a copy, and +you may at your option offer warranty protection in exchange for a fee. + + 2. You may modify your copy or copies of the Program or any portion +of it, thus forming a work based on the Program, and copy and +distribute such modifications or work under the terms of Section 1 +above, provided that you also meet all of these conditions: + + a) You must cause the modified files to carry prominent notices + stating that you changed the files and the date of any change. + + b) You must cause any work that you distribute or publish, that in + whole or in part contains or is derived from the Program or any + part thereof, to be licensed as a whole at no charge to all third + parties under the terms of this License. + + c) If the modified program normally reads commands interactively + when run, you must cause it, when started running for such + interactive use in the most ordinary way, to print or display an + announcement including an appropriate copyright notice and a + notice that there is no warranty (or else, saying that you provide + a warranty) and that users may redistribute the program under + these conditions, and telling the user how to view a copy of this + License. (Exception: if the Program itself is interactive but + does not normally print such an announcement, your work based on + the Program is not required to print an announcement.) + +These requirements apply to the modified work as a whole. If +identifiable sections of that work are not derived from the Program, +and can be reasonably considered independent and separate works in +themselves, then this License, and its terms, do not apply to those +sections when you distribute them as separate works. But when you +distribute the same sections as part of a whole which is a work based +on the Program, the distribution of the whole must be on the terms of +this License, whose permissions for other licensees extend to the +entire whole, and thus to each and every part regardless of who wrote it. + +Thus, it is not the intent of this section to claim rights or contest +your rights to work written entirely by you; rather, the intent is to +exercise the right to control the distribution of derivative or +collective works based on the Program. + +In addition, mere aggregation of another work not based on the Program +with the Program (or with a work based on the Program) on a volume of +a storage or distribution medium does not bring the other work under +the scope of this License. + + 3. You may copy and distribute the Program (or a work based on it, +under Section 2) in object code or executable form under the terms of +Sections 1 and 2 above provided that you also do one of the following: + + a) Accompany it with the complete corresponding machine-readable + source code, which must be distributed under the terms of Sections + 1 and 2 above on a medium customarily used for software interchange; or, + + b) Accompany it with a written offer, valid for at least three + years, to give any third party, for a charge no more than your + cost of physically performing source distribution, a complete + machine-readable copy of the corresponding source code, to be + distributed under the terms of Sections 1 and 2 above on a medium + customarily used for software interchange; or, + + c) Accompany it with the information you received as to the offer + to distribute corresponding source code. (This alternative is + allowed only for noncommercial distribution and only if you + received the program in object code or executable form with such + an offer, in accord with Subsection b above.) + +The source code for a work means the preferred form of the work for +making modifications to it. For an executable work, complete source +code means all the source code for all modules it contains, plus any +associated interface definition files, plus the scripts used to +control compilation and installation of the executable. However, as a +special exception, the source code distributed need not include +anything that is normally distributed (in either source or binary +form) with the major components (compiler, kernel, and so on) of the +operating system on which the executable runs, unless that component +itself accompanies the executable. + +If distribution of executable or object code is made by offering +access to copy from a designated place, then offering equivalent +access to copy the source code from the same place counts as +distribution of the source code, even though third parties are not +compelled to copy the source along with the object code. + + 4. You may not copy, modify, sublicense, or distribute the Program +except as expressly provided under this License. Any attempt +otherwise to copy, modify, sublicense or distribute the Program is +void, and will automatically terminate your rights under this License. +However, parties who have received copies, or rights, from you under +this License will not have their licenses terminated so long as such +parties remain in full compliance. + + 5. You are not required to accept this License, since you have not +signed it. However, nothing else grants you permission to modify or +distribute the Program or its derivative works. These actions are +prohibited by law if you do not accept this License. Therefore, by +modifying or distributing the Program (or any work based on the +Program), you indicate your acceptance of this License to do so, and +all its terms and conditions for copying, distributing or modifying +the Program or works based on it. + + 6. Each time you redistribute the Program (or any work based on the +Program), the recipient automatically receives a license from the +original licensor to copy, distribute or modify the Program subject to +these terms and conditions. You may not impose any further +restrictions on the recipients' exercise of the rights granted herein. +You are not responsible for enforcing compliance by third parties to +this License. + + 7. If, as a consequence of a court judgment or allegation of patent +infringement or for any other reason (not limited to patent issues), +conditions are imposed on you (whether by court order, agreement or +otherwise) that contradict the conditions of this License, they do not +excuse you from the conditions of this License. If you cannot +distribute so as to satisfy simultaneously your obligations under this +License and any other pertinent obligations, then as a consequence you +may not distribute the Program at all. For example, if a patent +license would not permit royalty-free redistribution of the Program by +all those who receive copies directly or indirectly through you, then +the only way you could satisfy both it and this License would be to +refrain entirely from distribution of the Program. + +If any portion of this section is held invalid or unenforceable under +any particular circumstance, the balance of the section is intended to +apply and the section as a whole is intended to apply in other +circumstances. + +It is not the purpose of this section to induce you to infringe any +patents or other property right claims or to contest validity of any +such claims; this section has the sole purpose of protecting the +integrity of the free software distribution system, which is +implemented by public license practices. Many people have made +generous contributions to the wide range of software distributed +through that system in reliance on consistent application of that +system; it is up to the author/donor to decide if he or she is willing +to distribute software through any other system and a licensee cannot +impose that choice. + +This section is intended to make thoroughly clear what is believed to +be a consequence of the rest of this License. + + 8. If the distribution and/or use of the Program is restricted in +certain countries either by patents or by copyrighted interfaces, the +original copyright holder who places the Program under this License +may add an explicit geographical distribution limitation excluding +those countries, so that distribution is permitted only in or among +countries not thus excluded. In such case, this License incorporates +the limitation as if written in the body of this License. + + 9. The Free Software Foundation may publish revised and/or new versions +of the General Public License from time to time. Such new versions will +be similar in spirit to the present version, but may differ in detail to +address new problems or concerns. + +Each version is given a distinguishing version number. If the Program +specifies a version number of this License which applies to it and "any +later version", you have the option of following the terms and conditions +either of that version or of any later version published by the Free +Software Foundation. If the Program does not specify a version number of +this License, you may choose any version ever published by the Free Software +Foundation. + + 10. If you wish to incorporate parts of the Program into other free +programs whose distribution conditions are different, write to the author +to ask for permission. For software which is copyrighted by the Free +Software Foundation, write to the Free Software Foundation; we sometimes +make exceptions for this. Our decision will be guided by the two goals +of preserving the free status of all derivatives of our free software and +of promoting the sharing and reuse of software generally. + + NO WARRANTY + + 11. BECAUSE THE PROGRAM IS LICENSED FREE OF CHARGE, THERE IS NO WARRANTY +FOR THE PROGRAM, TO THE EXTENT PERMITTED BY APPLICABLE LAW. EXCEPT WHEN +OTHERWISE STATED IN WRITING THE COPYRIGHT HOLDERS AND/OR OTHER PARTIES +PROVIDE THE PROGRAM "AS IS" WITHOUT WARRANTY OF ANY KIND, EITHER EXPRESSED +OR IMPLIED, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF +MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE. THE ENTIRE RISK AS +TO THE QUALITY AND PERFORMANCE OF THE PROGRAM IS WITH YOU. SHOULD THE +PROGRAM PROVE DEFECTIVE, YOU ASSUME THE COST OF ALL NECESSARY SERVICING, +REPAIR OR CORRECTION. + + 12. IN NO EVENT UNLESS REQUIRED BY APPLICABLE LAW OR AGREED TO IN WRITING +WILL ANY COPYRIGHT HOLDER, OR ANY OTHER PARTY WHO MAY MODIFY AND/OR +REDISTRIBUTE THE PROGRAM AS PERMITTED ABOVE, BE LIABLE TO YOU FOR DAMAGES, +INCLUDING ANY GENERAL, SPECIAL, INCIDENTAL OR CONSEQUENTIAL DAMAGES ARISING +OUT OF THE USE OR INABILITY TO USE THE PROGRAM (INCLUDING BUT NOT LIMITED +TO LOSS OF DATA OR DATA BEING RENDERED INACCURATE OR LOSSES SUSTAINED BY +YOU OR THIRD PARTIES OR A FAILURE OF THE PROGRAM TO OPERATE WITH ANY OTHER +PROGRAMS), EVEN IF SUCH HOLDER OR OTHER PARTY HAS BEEN ADVISED OF THE +POSSIBILITY OF SUCH DAMAGES. + + END OF TERMS AND CONDITIONS + + How to Apply These Terms to Your New Programs + + If you develop a new program, and you want it to be of the greatest +possible use to the public, the best way to achieve this is to make it +free software which everyone can redistribute and change under these terms. + + To do so, attach the following notices to the program. It is safest +to attach them to the start of each source file to most effectively +convey the exclusion of warranty; and each file should have at least +the "copyright" line and a pointer to where the full notice is found. + + + Copyright (C) + + This program is free software; you can redistribute it and/or modify + it under the terms of the GNU General Public License as published by + the Free Software Foundation; either version 2 of the License, or + (at your option) any later version. + + This program is distributed in the hope that it will be useful, + but WITHOUT ANY WARRANTY; without even the implied warranty of + MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the + GNU General Public License for more details. + + You should have received a copy of the GNU General Public License + along with this program; if not, write to the Free Software + Foundation, Inc., 59 Temple Place, Suite 330, Boston, MA 02111-1307 USA + + +Also add information on how to contact you by electronic and paper mail. + +If the program is interactive, make it output a short notice like this +when it starts in an interactive mode: + + Gnomovision version 69, Copyright (C) year name of author + Gnomovision comes with ABSOLUTELY NO WARRANTY; for details type `show w'. + This is free software, and you are welcome to redistribute it + under certain conditions; type `show c' for details. + +The hypothetical commands `show w' and `show c' should show the appropriate +parts of the General Public License. Of course, the commands you use may +be called something other than `show w' and `show c'; they could even be +mouse-clicks or menu items--whatever suits your program. + +You should also get your employer (if you work as a programmer) or your +school, if any, to sign a "copyright disclaimer" for the program, if +necessary. Here is a sample; alter the names: + + Yoyodyne, Inc., hereby disclaims all copyright interest in the program + `Gnomovision' (which makes passes at compilers) written by James Hacker. + + , 1 April 1989 + Ty Coon, President of Vice + +This General Public License does not permit incorporating your program into +proprietary programs. If your program is a subroutine library, you may +consider it more useful to permit linking proprietary applications with the +library. If this is what you want to do, use the GNU Library General +Public License instead of this License. + +-------------------------------------------------------------------------------- +11. ConcurrentQueue + + +This license file applies to everything in this repository except that which is explicitly annotated as being written by other authors, i.e. the Boost queue (included in the benchmarks for comparison), Intel's TBB library (ditto), the CDSChecker tool (used for verification), the Relacy model checker (ditto), and Jeff Preshing's semaphore implementation (used in the blocking queue) which has a zlib license (embedded in lightweightsempahore.h). + +Simplified BSD License: + +Copyright (c) 2013-2016, Cameron Desrochers. All rights reserved. + +Redistribution and use in source and binary forms, with or without modification, are permitted provided that the following conditions are met: + +Redistributions of source code must retain the above copyright notice, this list of conditions and the following disclaimer. +Redistributions in binary form must reproduce the above copyright notice, this list of conditions and the following disclaimer in the documentation and/or other materials provided with the distribution. +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +I have also chosen to dual-license under the Boost Software License as an alternative to the Simplified BSD license above: + +Boost Software License - Version 1.0 - August 17th, 2003 + +Permission is hereby granted, free of charge, to any person or organization obtaining a copy of the software and accompanying documentation covered by this license (the "Software") to use, reproduce, display, distribute, execute, and transmit the Software, and to prepare derivative works of the Software, and to permit third-parties to whom the Software is furnished to do so, all subject to the following: + +The copyright notices in the Software and this entire statement, including the above license grant, this restriction and the following disclaimer, must be included in all copies of the Software, in whole or in part, and all derivative works of the Software, unless such copies or derivative works are solely in the form of machine-executable object code generated by a source language processor. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE, TITLE AND NON-INFRINGEMENT. IN NO EVENT SHALL THE COPYRIGHT HOLDERS OR ANYONE DISTRIBUTING THE SOFTWARE BE LIABLE FOR ANY DAMAGES OR OTHER LIABILITY, WHETHER IN CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + +-------------------------------------------------------------------------------- +12. Cyan4973/xxHash + xxHash Library + Copyright (c) 2012-2020 Yann Collet + All rights reserved. + +BSD 2-Clause License (https://www.opensource.org/licenses/bsd-license.php) + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +* Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +* Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +---------------------------------------------------- + +xxhsum command line interface +Copyright (c) 2013-2020 Yann Collet +All rights reserved. + +GPL v2 License + +This program is free software; you can redistribute it and/or modify +it under the terms of the GNU General Public License as published by +the Free Software Foundation; either version 2 of the License, or +(at your option) any later version. + +This program is distributed in the hope that it will be useful, +but WITHOUT ANY WARRANTY; without even the implied warranty of +MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +GNU General Public License for more details. + +You should have received a copy of the GNU General Public License along +with this program; if not, write to the Free Software Foundation, Inc., +51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA. + +-------------------------------------------------------------------------------- +13. google-sparsehash + Copyright (c) 2005, Google Inc. + All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + * Neither the name of Google Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- +14. Memkind + +Unless otherwise specified, files in the memkind source distribution are +subject to the following license: + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of Intel Corporation nor the names of its contributors + may be used to endorse or promote products derived from this software + without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE LIABLE +FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR +SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER +CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, +OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- +15. Vmemcache + Copyright 2018-2019, Intel Corporation + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions + are met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in + the documentation and/or other materials provided with the + distribution. + + * Neither the name of the copyright holder nor the names of its + contributors may be used to endorse or promote products derived + from this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + Everything in this source tree is covered by the previous license + with the following exceptions: + + * src/fast_hash.c and src/fash_hash.h licensed unded MIT. + + * utils/cstyle (used only during development) licensed under CDDL. + +-------------------------------------------------------------------------------- +16. cpp-TimSort + Copyright (c) 2011 Fuji Goro (gfx) . + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + +------------------------------------------------------------- +The following third party programs have their own third party programs. These additional third party program files are as follows: + 1. Intel® oneAPI Data Analytics Library (oneDAL): third-party-programs-oneDAL.txt file + 2. Intel® oneAPI Collective Communications Library (oneCCL): third-party-programs-oneCCL.txt file +------------------------------------------------------------- + +*Other names and brands may be claimed as the property of others. + +------------------------------------------------------------- diff --git a/arrow-data-source/LICENSE.txt b/arrow-data-source/LICENSE.txt deleted file mode 100644 index 261eeb9e9..000000000 --- a/arrow-data-source/LICENSE.txt +++ /dev/null @@ -1,201 +0,0 @@ - Apache License - Version 2.0, January 2004 - http://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - APPENDIX: How to apply the Apache License to your work. - - To apply the Apache License to your work, attach the following - boilerplate notice, with the fields enclosed by brackets "[]" - replaced with your own identifying information. (Don't include - the brackets!) The text should be enclosed in the appropriate - comment syntax for the file format. We also recommend that a - file or class name and description of purpose be included on the - same "printed page" as the copyright notice for easier - identification within third-party archives. - - Copyright [yyyy] [name of copyright owner] - - 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. diff --git a/arrow-data-source/README.md b/arrow-data-source/README.md index 4fdfa8bd4..4d097f896 100644 --- a/arrow-data-source/README.md +++ b/arrow-data-source/README.md @@ -6,10 +6,6 @@ A Spark DataSource implementation for reading files into Arrow compatible column The development of this library is still in progress. As a result some of the functionality may not be constantly stable for being used in production environments that have not been fully considered due to the limited testing capabilities so far. -## Online Documentation - -You can find the all the Native SQL Engine documents on the [project web page](https://oap-project.github.io/arrow-data-source/). - ## Build ### Prerequisite @@ -17,17 +13,17 @@ You can find the all the Native SQL Engine documents on the [project web page](h There are some requirements before you build the project. Please make sure you have already installed the software in your system. -1. gcc 9.3 or higher version +1. GCC 7.0 or higher version 2. java8 OpenJDK -> yum install java-1.8.0-openjdk -3. cmake 3.2 or higher version -4. maven 3.1.1 or higher version +3. cmake 3.16 or higher version +4. maven 3.6 or higher version 5. Hadoop 2.7.5 or higher version 6. Spark 3.0.0 or higher version 7. Intel Optimized Arrow 3.0.0 ### Building by Conda -If you already have a working Hadoop Spark Cluster, we provide a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md) for more information. Once finished [OAP-Installation-Guide](./docs/OAP-Installation-Guide.md), you can find built `spark-arrow-datasource-standard--jar-with-dependencies.jar` under `$HOME/miniconda2/envs/oapenv/oap_jars`. +If you already have a working Hadoop Spark Cluster, we provide a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](../docs/OAP-Installation-Guide.md) for more information. Once finished [OAP-Installation-Guide](../docs/OAP-Installation-Guide.md), you can find built `spark-arrow-datasource-standard--jar-with-dependencies.jar` under `$HOME/miniconda2/envs/oapenv/oap_jars`. Then you can just skip steps below and jump to [Get Started](#get-started). ### cmake installation @@ -35,7 +31,7 @@ Then you can just skip steps below and jump to [Get Started](#get-started). If you are facing some trouble when installing cmake, please follow below steps to install cmake. ``` -// installing cmake 3.2 +// installing cmake 3.16.1 sudo yum install cmake3 // If you have an existing cmake, you can use below command to set it as an option within alternatives command @@ -121,7 +117,7 @@ You have to use a customized Arrow to support for our datasets Java API. ``` // build arrow-cpp -git clone -b https://github.com/Intel-bigdata/arrow.git +git clone -b arrow-3.0.0-oap-1.1 https://github.com/oap-project/arrow.git cd arrow/cpp mkdir build cd build @@ -213,7 +209,7 @@ spark.sql("SELECT * FROM my_temp_view LIMIT 10").show(10) To validate if ArrowDataSource works, you can go to the DAG to check if ArrowScan has been used from the above example query. -![Image of ArrowDataSource Validation](./docs/image/arrowdatasource_validation.png) +![Image of ArrowDataSource Validation](../docs/image/arrowdatasource_validation.png) ## Work together with ParquetDataSource (experimental) diff --git a/arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java b/arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java index b89e74fb6..f65e2057a 100644 --- a/arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java +++ b/arrow-data-source/common/src/main/java/com/intel/oap/vectorized/ArrowWritableColumnVector.java @@ -681,6 +681,11 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { writer.setFloats(rowId, count, src, srcIndex); } + @Override + public void putFloatsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + + } + @Override public float getFloat(int rowId) { return accessor.getFloat(rowId); @@ -710,6 +715,11 @@ public void putDoubles(int rowId, int count, double[] src, int srcIndex) { writer.setDoubles(rowId, count, src, srcIndex); } + @Override + public void putDoublesLittleEndian(int rowId, int count, byte[] src, int srcIndex) { + + } + @Override public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { writer.setDoubles(rowId, count, src, srcIndex); @@ -1571,6 +1581,12 @@ final void setLongs(int rowId, int count, byte[] src, int srcIndex) { } } + @Override + final void setDouble(int rowId, double value) { + long val = (long)value; + writer.setSafe(rowId, val); + } + @Override void setLongsLittleEndian(int rowId, int count, byte[] src, int srcIndex) { int srcOffset = srcIndex + Platform.BYTE_ARRAY_OFFSET; diff --git a/arrow-data-source/docs/ApacheArrowInstallation.md b/arrow-data-source/docs/ApacheArrowInstallation.md deleted file mode 100644 index 06cee2312..000000000 --- a/arrow-data-source/docs/ApacheArrowInstallation.md +++ /dev/null @@ -1,70 +0,0 @@ -# llvm-7.0: -Arrow Gandiva depends on LLVM, and I noticed current version strictly depends on llvm7.0 if you installed any other version rather than 7.0, it will fail. -``` shell -wget http://releases.llvm.org/7.0.1/llvm-7.0.1.src.tar.xz -tar xf llvm-7.0.1.src.tar.xz -cd llvm-7.0.1.src/ -cd tools -wget http://releases.llvm.org/7.0.1/cfe-7.0.1.src.tar.xz -tar xf cfe-7.0.1.src.tar.xz -mv cfe-7.0.1.src clang -cd .. -mkdir build -cd build -cmake .. -DCMAKE_BUILD_TYPE=Release -cmake --build . -j -cmake --build . --target install -# check if clang has also been compiled, if no -cd tools/clang -mkdir build -cd build -cmake .. -make -j -make install -``` - -# cmake: -Arrow will download package during compiling, in order to support SSL in cmake, build cmake is optional. -``` shell -wget https://github.com/Kitware/CMake/releases/download/v3.15.0-rc4/cmake-3.15.0-rc4.tar.gz -tar xf cmake-3.15.0-rc4.tar.gz -cd cmake-3.15.0-rc4/ -./bootstrap --system-curl --parallel=64 #parallel num depends on your server core number -make -j -make install -cmake --version -cmake version 3.15.0-rc4 -``` - -# Apache Arrow -``` shell -git clone https://github.com/Intel-bigdata/arrow.git -cd arrow && git checkout branch-0.17.0-oap-1.0 -mkdir -p arrow/cpp/release-build -cd arrow/cpp/release-build -cmake -DARROW_DEPENDENCY_SOURCE=BUNDLED -DARROW_GANDIVA_JAVA=ON -DARROW_GANDIVA=ON -DARROW_PARQUET=ON -DARROW_CSV=ON -DARROW_HDFS=ON -DARROW_BOOST_USE_SHARED=ON -DARROW_JNI=ON -DARROW_DATASET=ON -DARROW_WITH_PROTOBUF=ON -DARROW_WITH_SNAPPY=ON -DARROW_WITH_LZ4=ON -DARROW_FILESYSTEM=ON -DARROW_JSON=ON .. -make -j -make install - -# build java -cd ../../java -# change property 'arrow.cpp.build.dir' to the relative path of cpp build dir in gandiva/pom.xml -mvn clean install -P arrow-jni -am -Darrow.cpp.build.dir=../cpp/release-build/release/ -DskipTests -# if you are behine proxy, please also add proxy for socks -mvn clean install -P arrow-jni -am -Darrow.cpp.build.dir=../cpp/release-build/release/ -DskipTests -DsocksProxyHost=${proxyHost} -DsocksProxyPort=1080 -``` - -run test -``` shell -mvn test -pl adapter/parquet -P arrow-jni -mvn test -pl gandiva -P arrow-jni -``` - -# Copy binary files to oap-native-sql resources directory -Because oap-native-sql plugin will build a stand-alone jar file with arrow dependency, if you choose to build Arrow by yourself, you have to copy below files as a replacement from the original one. -You can find those files in Apache Arrow installation directory or release directory. Below example assume Apache Arrow has been installed on /usr/local/lib64 -``` shell -cp /usr/local/lib64/libarrow.so.17 $native-sql-engine-dir/cpp/src/resources -cp /usr/local/lib64/libgandiva.so.17 $native-sql-engine-dir/cpp/src/resources -cp /usr/local/lib64/libparquet.so.17 $native-sql-engine-dir/cpp/src/resources -``` diff --git a/arrow-data-source/docs/Configuration.md b/arrow-data-source/docs/Configuration.md deleted file mode 100644 index b20b46f0e..000000000 --- a/arrow-data-source/docs/Configuration.md +++ /dev/null @@ -1,29 +0,0 @@ -# Spark Configurations for Native SQL Engine - -Add below configuration to spark-defaults.conf - -``` -##### Columnar Process Configuration - -spark.sql.sources.useV1SourceList avro -spark.sql.join.preferSortMergeJoin false -spark.sql.extensions com.intel.oap.ColumnarPlugin -spark.shuffle.manager org.apache.spark.shuffle.sort.ColumnarShuffleManager - -# note native sql engine depends on arrow data source -spark.driver.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/spark-columnar-core--jar-with-dependencies.jar:$HOME/miniconda2/envs/oapenv/oap_jars/spark-arrow-datasource-standard--jar-with-dependencies.jar -spark.executor.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/spark-columnar-core--jar-with-dependencies.jar:$HOME/miniconda2/envs/oapenv/oap_jars/spark-arrow-datasource-standard--jar-with-dependencies.jar - -spark.executorEnv.LIBARROW_DIR $HOME/miniconda2/envs/oapenv -spark.executorEnv.CC $HOME/miniconda2/envs/oapenv/bin/gcc -###### -``` - -Before you start spark, you must use below command to add some environment variables. - -``` -export CC=$HOME/miniconda2/envs/oapenv/bin/gcc -export LIBARROW_DIR=$HOME/miniconda2/envs/oapenv/ -``` - -About arrow-data-source.jar, you can refer [Unified Arrow Data Source ](https://oap-project.github.io/arrow-data-source/). diff --git a/arrow-data-source/docs/Installation.md b/arrow-data-source/docs/Installation.md deleted file mode 100644 index 1b172ba50..000000000 --- a/arrow-data-source/docs/Installation.md +++ /dev/null @@ -1,27 +0,0 @@ -# Spark Native SQL Engine Installation - -For detailed testing scripts, please refer to [solution guide](https://github.com/Intel-bigdata/Solution_navigator/tree/master/nativesql) - -## Install Googletest and Googlemock - -``` shell -yum install gtest-devel -yum install gmock -``` - -## Build Native SQL Engine - -cmake parameters: -BUILD_ARROW(Default is On): Build Arrow from Source -STATIC_ARROW(Default is Off): When BUILD_ARROW is ON, you can choose to build static or shared Arrow library, please notice current only support to build SHARED ARROW. -ARROW_ROOT(Default is /usr/local): When BUILD_ARROW is OFF, you can set the ARROW library path to link the existing library in your environment. -BUILD_PROTOBUF(Default is On): Build Protobuf from Source - -``` shell -git clone -b ${version} https://github.com/oap-project/native-sql-engine.git -cd native-sql-engine -mvn clean package -am -DskipTests -Dcpp_tests=OFF -Dbuild_arrow=ON -Dstatic_arrow=OFF -Darrow_root=/usr/local -Dbuild_protobuf=ON -``` - -### Additonal Notes -[Notes for Installation Issues](./InstallationNotes.md) diff --git a/arrow-data-source/docs/InstallationNotes.md b/arrow-data-source/docs/InstallationNotes.md deleted file mode 100644 index cf7120be9..000000000 --- a/arrow-data-source/docs/InstallationNotes.md +++ /dev/null @@ -1,47 +0,0 @@ -### Notes for Installation Issues -* Before the Installation, if you have installed other version of oap-native-sql, remove all installed lib and include from system path: libarrow* libgandiva* libspark-columnar-jni* - -* libgandiva_jni.so was not found inside JAR - -change property 'arrow.cpp.build.dir' to $ARROW_DIR/cpp/release-build/release/ in gandiva/pom.xml. If you do not want to change the contents of pom.xml, specify it like this: - -``` -mvn clean install -P arrow-jni -am -Darrow.cpp.build.dir=/root/git/t/arrow/cpp/release-build/release/ -DskipTests -Dcheckstyle.skip -``` - -* No rule to make target '../src/protobuf_ep', needed by `src/proto/Exprs.pb.cc' - -remove the existing libprotobuf installation, then the script for find_package() will be able to download protobuf. - -* can't find the libprotobuf.so.13 in the shared lib - -copy the libprotobuf.so.13 from $OAP_DIR/oap-native-sql/cpp/src/resources to /usr/lib64/ - -* unable to load libhdfs: libgsasl.so.7: cannot open shared object file - -libgsasl is missing, run `yum install libgsasl` - -* CentOS 7.7 looks like didn't provide the glibc we required, so binaries packaged on F30 won't work. - -``` -20/04/21 17:46:17 WARN TaskSetManager: Lost task 0.1 in stage 1.0 (TID 2, 10.0.0.143, executor 6): java.lang.UnsatisfiedLinkError: /tmp/libgandiva_jni.sobe729912-3bbe-4bd0-bb96-4c7ce2e62336: /lib64/libm.so.6: version `GLIBC_2.29' not found (required by /tmp/libgandiva_jni.sobe729912-3bbe-4bd0-bb96-4c7ce2e62336) -``` - -* Missing symbols due to old GCC version. - -``` -[root@vsr243 release-build]# nm /usr/local/lib64/libparquet.so | grep ZN5boost16re_detail_10710012perl_matcherIN9__gnu_cxx17__normal_iteratorIPKcSsEESaINS_9sub_matchIS6_EEENS_12regex_traitsIcNS_16cpp_regex_traitsIcEEEEE14construct_initERKNS_11basic_regexIcSD_EENS_15regex_constants12_match_flagsE -_ZN5boost16re_detail_10710012perl_matcherIN9__gnu_cxx17__normal_iteratorIPKcSsEESaINS_9sub_matchIS6_EEENS_12regex_traitsIcNS_16cpp_regex_traitsIcEEEEE14construct_initERKNS_11basic_regexIcSD_EENS_15regex_constants12_match_flagsE -``` - -Need to compile all packags with newer GCC: - -``` -[root@vsr243 ~]# export CXX=/usr/local/bin/g++ -[root@vsr243 ~]# export CC=/usr/local/bin/gcc -``` - -* Can not connect to hdfs @sr602 - -vsr606, vsr243 are both not able to connect to hdfs @sr602, need to skipTests to generate the jar - diff --git a/arrow-data-source/docs/OAP-Developer-Guide.md b/arrow-data-source/docs/OAP-Developer-Guide.md deleted file mode 100644 index 8d7ac6abf..000000000 --- a/arrow-data-source/docs/OAP-Developer-Guide.md +++ /dev/null @@ -1,109 +0,0 @@ -# OAP Developer Guide - -This document contains the instructions & scripts on installing necessary dependencies and building OAP. -You can get more detailed information from OAP each module below. - -* [SQL Index and Data Source Cache](https://github.com/oap-project/sql-ds-cache/blob/master/docs/Developer-Guide.md) -* [PMem Common](https://github.com/oap-project/pmem-common) -* [PMem Shuffle](https://github.com/oap-project/pmem-shuffle#5-install-dependencies-for-shuffle-remote-pmem-extension) -* [Remote Shuffle](https://github.com/oap-project/remote-shuffle) -* [OAP MLlib](https://github.com/oap-project/oap-mllib) -* [Arrow Data Source](https://github.com/oap-project/arrow-data-source) -* [Native SQL Engine](https://github.com/oap-project/native-sql-engine) - -## Building OAP - -### Prerequisites for Building - -OAP is built with [Apache Maven](http://maven.apache.org/) and Oracle Java 8, and mainly required tools to install on your cluster are listed below. - -- [Cmake](https://help.directadmin.com/item.php?id=494) -- [GCC > 7](https://gcc.gnu.org/wiki/InstallingGCC) -- [Memkind](https://github.com/memkind/memkind/tree/v1.10.1-rc2) -- [Vmemcache](https://github.com/pmem/vmemcache) -- [HPNL](https://github.com/Intel-bigdata/HPNL) -- [PMDK](https://github.com/pmem/pmdk) -- [OneAPI](https://software.intel.com/content/www/us/en/develop/tools/oneapi.html) -- [Arrow](https://github.com/Intel-bigdata/arrow) - -- **Requirements for Shuffle Remote PMem Extension** -If enable Shuffle Remote PMem extension with RDMA, you can refer to [PMem Shuffle](https://github.com/oap-project/pmem-shuffle) to configure and validate RDMA in advance. - -We provide scripts below to help automatically install dependencies above **except RDMA**, need change to **root** account, run: - -``` -# git clone -b https://github.com/Intel-bigdata/OAP.git -# cd OAP -# sh $OAP_HOME/dev/install-compile-time-dependencies.sh -``` - -Run the following command to learn more. - -``` -# sh $OAP_HOME/dev/scripts/prepare_oap_env.sh --help -``` - -Run the following command to automatically install specific dependency such as Maven. - -``` -# sh $OAP_HOME/dev/scripts/prepare_oap_env.sh --prepare_maven -``` - - -### Building - -To build OAP package, run command below then you can find a tarball named `oap-$VERSION-bin-spark-$VERSION.tar.gz` under directory `$OAP_HOME/dev/release-package `. -``` -$ sh $OAP_HOME/dev/compile-oap.sh -``` - -Building Specified OAP Module, such as `oap-cache`, run: -``` -$ sh $OAP_HOME/dev/compile-oap.sh --oap-cache -``` - - -### Running OAP Unit Tests - -Setup building environment manually for intel MLlib, and if your default GCC version is before 7.0 also need export `CC` & `CXX` before using `mvn`, run - -``` -$ export CXX=$OAP_HOME/dev/thirdparty/gcc7/bin/g++ -$ export CC=$OAP_HOME/dev/thirdparty/gcc7/bin/gcc -$ export ONEAPI_ROOT=/opt/intel/inteloneapi -$ source /opt/intel/inteloneapi/daal/2021.1-beta07/env/vars.sh -$ source /opt/intel/inteloneapi/tbb/2021.1-beta07/env/vars.sh -$ source /tmp/oneCCL/build/_install/env/setvars.sh -``` - -Run all the tests: - -``` -$ mvn clean test -``` - -Run Specified OAP Module Unit Test, such as `oap-cache`: - -``` -$ mvn clean -pl com.intel.oap:oap-cache -am test - -``` - -### Building SQL Index and Data Source Cache with PMem - -#### Prerequisites for building with PMem support - -When using SQL Index and Data Source Cache with PMem, finish steps of [Prerequisites for building](#prerequisites-for-building) to ensure needed dependencies have been installed. - -#### Building package - -You can build OAP with PMem support with command below: - -``` -$ sh $OAP_HOME/dev/compile-oap.sh -``` -Or run: - -``` -$ mvn clean -q -Ppersistent-memory -Pvmemcache -DskipTests package -``` diff --git a/arrow-data-source/docs/OAP-Installation-Guide.md b/arrow-data-source/docs/OAP-Installation-Guide.md deleted file mode 100644 index e3b229805..000000000 --- a/arrow-data-source/docs/OAP-Installation-Guide.md +++ /dev/null @@ -1,69 +0,0 @@ -# OAP Installation Guide -This document introduces how to install OAP and its dependencies on your cluster nodes by ***Conda***. -Follow steps below on ***every node*** of your cluster to set right environment for each machine. - -## Contents - - [Prerequisites](#prerequisites) - - [Installing OAP](#installing-oap) - - [Configuration](#configuration) - -## Prerequisites - -- **OS Requirements** -We have tested OAP on Fedora 29 and CentOS 7.6 (kernel-4.18.16). We recommend you use **Fedora 29 CentOS 7.6 or above**. Besides, for [Memkind](https://github.com/memkind/memkind/tree/v1.10.1-rc2) we recommend you use **kernel above 3.10**. - -- **Conda Requirements** -Install Conda on your cluster nodes with below commands and follow the prompts on the installer screens.: -```bash -$ wget -c https://repo.continuum.io/miniconda/Miniconda2-latest-Linux-x86_64.sh -$ chmod +x Miniconda2-latest-Linux-x86_64.sh -$ bash Miniconda2-latest-Linux-x86_64.sh -``` -For changes to take effect, close and re-open your current shell. To test your installation, run the command `conda list` in your terminal window. A list of installed packages appears if it has been installed correctly. - -## Installing OAP - -Dependencies below are required by OAP and all of them are included in OAP Conda package, they will be automatically installed in your cluster when you Conda install OAP. Ensure you have activated environment which you created in the previous steps. - -- [Arrow](https://github.com/Intel-bigdata/arrow) -- [Plasma](http://arrow.apache.org/blog/2017/08/08/plasma-in-memory-object-store/) -- [Memkind](https://anaconda.org/intel/memkind) -- [Vmemcache](https://anaconda.org/intel/vmemcache) -- [HPNL](https://anaconda.org/intel/hpnl) -- [PMDK](https://github.com/pmem/pmdk) -- [OneAPI](https://software.intel.com/content/www/us/en/develop/tools/oneapi.html) - - -Create a conda environment and install OAP Conda package. -```bash -$ conda create -n oapenv -y python=3.7 -$ conda activate oapenv -$ conda install -c conda-forge -c intel -y oap=1.0.0 -``` - -Once finished steps above, you have completed OAP dependencies installation and OAP building, and will find built OAP jars under `$HOME/miniconda2/envs/oapenv/oap_jars` - -#### Extra Steps for Shuffle Remote PMem Extension - -If you use one of OAP features -- [PMmem Shuffle](https://github.com/oap-project/pmem-shuffle) with **RDMA**, you need to configure and validate RDMA, please refer to [PMem Shuffle](https://github.com/oap-project/pmem-shuffle#4-configure-and-validate-rdma) for the details. - - -## Configuration - -Once finished steps above, make sure libraries installed by Conda can be linked by Spark, please add the following configuration settings to `$SPARK_HOME/conf/spark-defaults.conf`. - -``` -spark.executorEnv.LD_LIBRARY_PATH $HOME/miniconda2/envs/oapenv/lib -spark.executor.extraLibraryPath $HOME/miniconda2/envs/oapenv/lib -spark.driver.extraLibraryPath $HOME/miniconda2/envs/oapenv/lib -spark.executor.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/$OAP_FEATURE.jar -spark.driver.extraClassPath $HOME/miniconda2/envs/oapenv/oap_jars/$OAP_FEATURE.jar -``` - -And then you can follow the corresponding feature documents for more details to use them. - - - - - - diff --git a/arrow-data-source/docs/Prerequisite.md b/arrow-data-source/docs/Prerequisite.md deleted file mode 100644 index 5ff82aa1b..000000000 --- a/arrow-data-source/docs/Prerequisite.md +++ /dev/null @@ -1,151 +0,0 @@ -# Prerequisite - -There are some requirements before you build the project. -Please make sure you have already installed the software in your system. - -1. gcc 9.3 or higher version -2. java8 OpenJDK -> yum install java-1.8.0-openjdk -3. cmake 3.2 or higher version -4. maven 3.1.1 or higher version -5. Hadoop 2.7.5 or higher version -6. Spark 3.0.0 or higher version -7. Intel Optimized Arrow 0.17.0 - -## gcc installation - -// installing gcc 9.3 or higher version - -Please notes for better performance support, gcc 9.3 is a minimal requirement with Intel Microarchitecture such as SKYLAKE, CASCADELAKE, ICELAKE. -https://gcc.gnu.org/install/index.html - -Follow the above website to download gcc. -C++ library may ask a certain version, if you are using gcc 9.3 the version would be libstdc++.so.6.0.28. -You may have to launch ./contrib/download_prerequisites command to install all the prerequisites for gcc. -If you are facing downloading issue in download_prerequisites command, you can try to change ftp to http. - -//Follow the steps to configure gcc -https://gcc.gnu.org/install/configure.html - -If you are facing a multilib issue, you can try to add --disable-multilib parameter in ../configure - -//Follow the steps to build gc -https://gcc.gnu.org/install/build.html - -//Follow the steps to install gcc -https://gcc.gnu.org/install/finalinstall.html - -//Set up Environment for new gcc -``` -export PATH=$YOUR_GCC_INSTALLATION_DIR/bin:$PATH -export LD_LIBRARY_PATH=$YOUR_GCC_INSTALLATION_DIR/lib64:$LD_LIBRARY_PATH -``` -Please remember to add and source the setup in your environment files such as /etc/profile or /etc/bashrc - -//Verify if gcc has been installation -Use gcc -v command to verify if your gcc version is correct.(Must larger than 9.3) - -## cmake installation -If you are facing some trouble when installing cmake, please follow below steps to install cmake. - -``` -// installing cmake 3.2 -sudo yum install cmake3 - -// If you have an existing cmake, you can use below command to set it as an option within alternatives command -sudo alternatives --install /usr/local/bin/cmake cmake /usr/bin/cmake 10 --slave /usr/local/bin/ctest ctest /usr/bin/ctest --slave /usr/local/bin/cpack cpack /usr/bin/cpack --slave /usr/local/bin/ccmake ccmake /usr/bin/ccmake --family cmake - -// Set cmake3 as an option within alternatives command -sudo alternatives --install /usr/local/bin/cmake cmake /usr/bin/cmake3 20 --slave /usr/local/bin/ctest ctest /usr/bin/ctest3 --slave /usr/local/bin/cpack cpack /usr/bin/cpack3 --slave /usr/local/bin/ccmake ccmake /usr/bin/ccmake3 --family cmake - -// Use alternatives to choose cmake version -sudo alternatives --config cmake -``` - -## maven installation - -If you are facing some trouble when installing maven, please follow below steps to install maven - -// installing maven 3.6.3 - -Go to https://maven.apache.org/download.cgi and download the specific version of maven - -// Below command use maven 3.6.3 as an example -``` -wget htps://ftp.wayne.edu/apache/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz -wget https://ftp.wayne.edu/apache/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz -tar xzf apache-maven-3.6.3-bin.tar.gz -mkdir /usr/local/maven -mv apache-maven-3.6.3/ /usr/local/maven/ -``` - -// Set maven 3.6.3 as an option within alternatives command -``` -sudo alternatives --install /usr/bin/mvn mvn /usr/local/maven/apache-maven-3.6.3/bin/mvn 1 -``` - -// Use alternatives to choose mvn version - -``` -sudo alternatives --config mvn -``` - -## HADOOP/SPARK Installation - -If there is no existing Hadoop/Spark installed, Please follow the guide to install your Hadoop/Spark [SPARK/HADOOP Installation](./SparkInstallation.md) - -### Hadoop Native Library(Default) - -Please make sure you have set up Hadoop directory properly with Hadoop Native Libraries -By default, Apache Arrow would scan `$HADOOP_HOME` and find the native Hadoop library `libhdfs.so`(under `$HADOOP_HOME/lib/native` directory) to be used for Hadoop client. - -You can also use `ARROW_LIBHDFS_DIR` to configure the location of `libhdfs.so` if it is installed in other directory than `$HADOOP_HOME/lib/native` - -If your SPARK and HADOOP are separated in different nodes, please find `libhdfs.so` in your Hadoop cluster and copy it to SPARK cluster, then use one of the above methods to set it properly. - -For more information, please check -Arrow HDFS interface [documentation](https://github.com/apache/arrow/blob/master/cpp/apidoc/HDFS.md) -Hadoop Native Library, please read the official Hadoop website [documentation](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-common/NativeLibraries.html) - -### Use libhdfs3 library for better performance(Optional) - -For better performance ArrowDataSource reads HDFS files using the third-party library `libhdfs3`. The library must be pre-installed on machines Spark Executor nodes are running on. - -To install the library, use of [Conda](https://docs.conda.io/en/latest/) is recommended. - -``` -// installing libhdfs3 -conda install -c conda-forge libhdfs3 - -// check the installed library file -ll ~/miniconda/envs/$(YOUR_ENV_NAME)/lib/libhdfs3.so -``` - -We also provide a libhdfs3 binary in cpp/src/resources directory. - -To set up libhdfs3, there are two different ways: -Option1: Overwrite the soft link for libhdfs.so -To install libhdfs3.so, you have to create a soft link for libhdfs.so in your Hadoop directory(`$HADOOP_HOME/lib/native` by default). - -``` -ln -f -s libhdfs3.so libhdfs.so -``` - -Option2: -Add env variable to the system -``` -export ARROW_LIBHDFS3_DIR="PATH_TO_LIBHDFS3_DIR/" -``` - -Add following Spark configuration options before running the DataSource to make the library to be recognized: - -* `spark.executorEnv.ARROW_LIBHDFS3_DIR = "PATH_TO_LIBHDFS3_DIR/"` -* `spark.executorEnv.LD_LIBRARY_PATH = "PATH_TO_LIBHDFS3_DEPENDENCIES_DIR/"` - -Please notes: If you choose to use libhdfs3.so, there are some other dependency libraries you have to installed such as libprotobuf or libcrypto. - - -## Intel Optimized Apache Arrow Installation - -Intel Optimized Apache Arrow is MANDATORY to be used. However, we have a bundle a compiled arrow libraries(libarrow, libgandiva, libparquet) built by GCC9.3 included in the cpp/src/resources directory. -If you wish to build Apache Arrow by yourself, please follow the guide to build and install Apache Arrow [ArrowInstallation](./ApacheArrowInstallation.md) - diff --git a/arrow-data-source/docs/SparkInstallation.md b/arrow-data-source/docs/SparkInstallation.md deleted file mode 100644 index 9d2a864ae..000000000 --- a/arrow-data-source/docs/SparkInstallation.md +++ /dev/null @@ -1,44 +0,0 @@ -### Download Spark 3.0.1 - -Currently Native SQL Engine works on the Spark 3.0.1 version. - -``` -wget http://archive.apache.org/dist/spark/spark-3.0.1/spark-3.0.1-bin-hadoop3.2.tgz -sudo mkdir -p /opt/spark && sudo mv spark-3.0.1-bin-hadoop3.2.tgz /opt/spark -sudo cd /opt/spark && sudo tar -xf spark-3.0.1-bin-hadoop3.2.tgz -export SPARK_HOME=/opt/spark/spark-3.0.1-bin-hadoop3.2/ -``` - -### [Or building Spark from source](https://spark.apache.org/docs/latest/building-spark.html) - -``` shell -git clone https://github.com/intel-bigdata/spark.git -cd spark && git checkout native-sql-engine-clean -# check spark supported hadoop version -grep \ -r pom.xml - 2.7.4 - 3.2.0 -# so we should build spark specifying hadoop version as 3.2 -./build/mvn -Pyarn -Phadoop-3.2 -Dhadoop.version=3.2.0 -DskipTests clean install -``` -Specify SPARK_HOME to spark path - -``` shell -export SPARK_HOME=${HADOOP_PATH} -``` - -### Hadoop building from source - -``` shell -git clone https://github.com/apache/hadoop.git -cd hadoop -git checkout rel/release-3.2.0 -# only build binary for hadoop -mvn clean install -Pdist -DskipTests -Dtar -# build binary and native library such as libhdfs.so for hadoop -# mvn clean install -Pdist,native -DskipTests -Dtar -``` - -``` shell -export HADOOP_HOME=${HADOOP_PATH}/hadoop-dist/target/hadoop-3.2.0/ -``` diff --git a/arrow-data-source/docs/User-Guide.md b/arrow-data-source/docs/User-Guide.md deleted file mode 100644 index c3c05cebf..000000000 --- a/arrow-data-source/docs/User-Guide.md +++ /dev/null @@ -1,118 +0,0 @@ -# Spark Native SQL Engine - -A Native Engine for Spark SQL with vectorized SIMD optimizations - -## Introduction - -![Overview](./image/nativesql_arch.png) - -Spark SQL works very well with structured row-based data. It used WholeStageCodeGen to improve the performance by Java JIT code. However Java JIT is usually not working very well on utilizing latest SIMD instructions, especially under complicated queries. [Apache Arrow](https://arrow.apache.org/) provided CPU-cache friendly columnar in-memory layout, its SIMD optimized kernels and LLVM based SQL engine Gandiva are also very efficient. Native SQL Engine used these technoligies and brought better performance to Spark SQL. - -## Key Features - -### Apache Arrow formatted intermediate data among Spark operator - -![Overview](./image/columnar.png) - -With [Spark 27396](https://issues.apache.org/jira/browse/SPARK-27396) its possible to pass a RDD of Columnarbatch to operators. We implemented this API with Arrow columnar format. - -### Apache Arrow based Native Readers for Parquet and other formats - -![Overview](./image/dataset.png) - -A native parquet reader was developed to speed up the data loading. it's based on Apache Arrow Dataset. For details please check [Arrow Data Source](https://github.com/oap-project/arrow-data-source) - -### Apache Arrow Compute/Gandiva based operators - -![Overview](./image/kernel.png) - -We implemented common operators based on Apache Arrow Compute and Gandiva. The SQL expression was compiled to one expression tree with protobuf and passed to native kernels. The native kernels will then evaluate the these expressions based on the input columnar batch. - -### Native Columnar Shuffle Operator with efficient compression support - -![Overview](./image/shuffle.png) - -We implemented columnar shuffle to improve the shuffle performance. With the columnar layout we could do very efficient data compression for different data format. - -## Build the Plugin - -### Building by Conda - -If you already have a working Hadoop Spark Cluster, we provide a Conda package which will automatically install dependencies needed by OAP, you can refer to [OAP-Installation-Guide](./OAP-Installation-Guide.md) for more information. Once finished [OAP-Installation-Guide](./OAP-Installation-Guide.md), you can find built `spark-columnar-core--jar-with-dependencies.jar` under `$HOME/miniconda2/envs/oapenv/oap_jars`. -Then you can just skip below steps and jump to Getting Started [Get Started](#get-started). - -### Building by yourself - -If you prefer to build from the source code on your hand, please follow below steps to set up your environment. - -### Prerequisite -There are some requirements before you build the project. -Please check the document [Prerequisite](./Prerequisite.md) and make sure you have already installed the software in your system. -If you are running a SPARK Cluster, please make sure all the software are installed in every single node. - -### Installation -Please check the document [Installation Guide](./Installation.md) - -### Configuration & Testing -Please check the document [Configuration Guide](./Configuration.md) - -## Get started -To enable OAP NativeSQL Engine, the previous built jar `spark-columnar-core--jar-with-dependencies.jar` should be added to Spark configuration. We also recommend to use `spark-arrow-datasource-standard--jar-with-dependencies.jar`. We will demonstrate an example by using both jar files. -SPARK related options are: - -* `spark.driver.extraClassPath` : Set to load jar file to driver. -* `spark.executor.extraClassPath` : Set to load jar file to executor. -* `jars` : Set to copy jar file to the executors when using yarn cluster mode. -* `spark.executorEnv.ARROW_LIBHDFS3_DIR` : Optional if you are using a custom libhdfs3.so. -* `spark.executorEnv.LD_LIBRARY_PATH` : Optional if you are using a custom libhdfs3.so. - -For Spark Standalone Mode, please set the above value as relative path to the jar file. -For Spark Yarn Cluster Mode, please set the above value as absolute path to the jar file. - -Example to run Spark Shell with ArrowDataSource jar file -``` -${SPARK_HOME}/bin/spark-shell \ - --verbose \ - --master yarn \ - --driver-memory 10G \ - --conf spark.driver.extraClassPath=$PATH_TO_JAR/spark-arrow-datasource-standard--jar-with-dependencies.jar:$PATH_TO_JAR/spark-columnar-core--jar-with-dependencies.jar \ - --conf spark.executor.extraClassPath=$PATH_TO_JAR/spark-arrow-datasource-standard--jar-with-dependencies.jar:$PATH_TO_JAR/spark-columnar-core--jar-with-dependencies.jar \ - --conf spark.driver.cores=1 \ - --conf spark.executor.instances=12 \ - --conf spark.executor.cores=6 \ - --conf spark.executor.memory=20G \ - --conf spark.memory.offHeap.size=80G \ - --conf spark.task.cpus=1 \ - --conf spark.locality.wait=0s \ - --conf spark.sql.shuffle.partitions=72 \ - --conf spark.executorEnv.ARROW_LIBHDFS3_DIR="$PATH_TO_LIBHDFS3_DIR/" \ - --conf spark.executorEnv.LD_LIBRARY_PATH="$PATH_TO_LIBHDFS3_DEPENDENCIES_DIR" - --jars $PATH_TO_JAR/spark-arrow-datasource-standard--jar-with-dependencies.jar,$PATH_TO_JAR/spark-columnar-core--jar-with-dependencies.jar -``` - -Here is one example to verify if native sql engine works, make sure you have TPC-H dataset. We could do a simple projection on one parquet table. For detailed testing scripts, please refer to [Solution Guide](https://github.com/Intel-bigdata/Solution_navigator/tree/master/nativesql). -``` -val orders = spark.read.format("arrow").load("hdfs:////user/root/date_tpch_10/orders") -orders.createOrReplaceTempView("orders") -spark.sql("select * from orders where o_orderdate > date '1998-07-26'").show(20000, false) -``` - -The result should show up on Spark console and you can check the DAG diagram with some Columnar Processing stage. - - -## Performance data - -For initial microbenchmark performance, we add 10 fields up with spark, data size is 200G data - -![Performance](./image/performance.png) - -## Coding Style - -* For Java code, we used [google-java-format](https://github.com/google/google-java-format) -* For Scala code, we used [Spark Scala Format](https://github.com/apache/spark/blob/master/dev/.scalafmt.conf), please use [scalafmt](https://github.com/scalameta/scalafmt) or run ./scalafmt for scala codes format -* For Cpp codes, we used Clang-Format, check on this link [google-vim-codefmt](https://github.com/google/vim-codefmt) for details. - -## Contact - -chendi.xue@intel.com -binwei.yang@intel.com diff --git a/arrow-data-source/docs/image/columnar.png b/arrow-data-source/docs/image/columnar.png deleted file mode 100644 index d89074905638fba98e7eddc065e97b05a08f4316..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 43622 zcmd42cT|&2v_5J>L_kE8UX(<-l+dM#Na)gg6GDIhp@Vc(x`f_~Aieh394T}y|)Ro7MK!0hA7##^~io=ew6ZtY~buP+91M6 z%%>tm&V~`}2>(P1*=1`ZpysE3R1vZm#e(6Ppl}9;`{qQhzS7pr$b0r z8+iCnhh2z(u#Z@UUwv2}l0d)&!uJ#K4|b?bsukF_)B4VWzq?WdmRgjl9(g&5Jug@o3YP0H5d-UP+= zFN1A>{Y?w>%lLfW-sVSNEm~T_cMeGIbQRyyD|={d%ZJ$HCJh`)jfLt_ zq@<}*bA_z|S3P5oYQtc0w=voT*ijOI%|pr&eE=&r1AH%~AM_$IN|ij@1g2{rBGZ-F z|53XfNy0qqK-&c>32jePJ(qNkZV9iPtK}(i{NSiUc3u+K ze!5<zJ@7LZgBMIB{n=tk(isGdVO(fvMWn-FZP+)zES zc4uI46rIPpdS{EGT0$-oRso>jo{##~oG7 z?a?Amgy80eq@!{|N0DiJYS?9Kj8q`T@-;RbYE^8BkInW@jS)}_4;vqLBm#7FUOnA6 zbLvViMv`_b4K8w8^Z%@?R9wySdcZ3FI;ollRXY^KWT%$_NE_t!0zw0%>Rvngrnq5kYa#T)`#&q=4DH~A6@K`hL#8PF)si#F-x94Vw*qd~sI zp4ID>u7KANtvMElG;or|P<)UO56Ue{fb(6>;?aofQg>(ZI$9f@>xi zML!=^47lj}R+Cqi^5+Z+KtZuB1pN*+79*!!7VYH~A=cTJz3SrlW2K7F`RP)>$p`Mi zihD`7?nm&6Z2(sbo4BSiZ>EIc8ew_5V5b9Lu6QF71LGN ztEcqx@rAr_Uml7~p2mdZG5y82JoJ-vfIk^|nf24Ab!xPt2Lw2+LCs|N4O;(W@d>%f zE&h;% zIkjdZb@k9CJgawu^X-(DM0-RTs(0G&J`>IgPv`>7ZBjXK1ilz&t7;OsC#BmuR+=cY zVzy4ui%Dc|Z&@Diqc=<#ix+(G8kte^(7|B%aha$7NrVa-p}G^K^IqTJjZ694So_x6 z6_|R$1)YWr_O?kq7kXS}wp}+xLE`~xRl(Xpn2U-n&%Z`}u%=SU!K-R7l7KB8k1>=X zp1G|)v)DFGIyP&AJ^YLdn|_ktp8S+<#T=sdTJP}?Hu6Yv>$6i?Ep*)lZZUx}AF>AY zNL8iQ=i)JDf|jI=DYA%?P~TP7bANW`(bO5Ff}B6{%_g!H^=22%B7Z&$ClO@E{!q-p zP%+i0FOhD1MmSbrGVRC+b=;k-H!H6Yyn}}O!K&Tt$5D6z)`1|DxPy@d@Gfi}FhL1t z?^&gZ6$JEVwQzUs#`^SR8_H5S&)`Ab`1qNPw_q}0&=?lce z;Xk@c-gs*zbGmorFX18T$ZJ?opUc1H+o_7$FAC)oLw#}vEm0%RFP@cJRH5x>*FApF z$13p|+L0P=@PxIYJoVu}EDMWX4CXtupYy&=6RO?2%H5ecSV6y#Mvs@~R`gl2wGMX4yiK1`Y|_mY-YH5u z2q5<>URJU=Au*2&%l$29s&jmOScAriSl; zQucM`=^+xt0(CM|G7C6uA1pQ^q(;KH5w#UG7HN&kq*>wG{uLo2Q~3oXqwP8y9^p$R z#|chfLA;fsqwV}f8~SI$<0LX&ztJpZbpp=m)w9#ch{jD>jQEeAeonC$CZqM6S0RWP z{8(k|SbYuaaBbiN{UQK-<<&@&S>%JTw(<(<+Q$52r;^Vm?*8M~iFNGl<5u4f;x|&Q zHMiKhler(mXx<++aLBkHhUCY%t~bn6c));?BEgDmF|1jK+Ti!M?qpvhf-AIH8$ z^iQC=Yv7`$QIjev-7L9_QHli5FcBH@TPh(`@ors{TL60aF#`-q8kkYhW6Mctkydk$ zD5X`kyC@_#$T*~QO$e-XphSuKvHu&%xkCnFoMA@RqH6L8W|@^HwAH;vnr9ymh~;~} z|La<==UYTfP%V4<_@kp3mwW@`SOJ`aRwd@mG__j9`k^UZ5`NB>j|0Qv#tU=TpyXu? zlQmQ$foTMW=M!&gwJbW^Bg{6`Ph+`mU)<`;pbHWu^?F*u2_f@3}1)$ zA3Rui<0TVOEYq37l6#A`jE$toug6`k(p^bwlgB~~)JqQCp-u96Ah|qixh~Bm`E#LM zB)MgsivAhmVIZhd6oc@#87I_=T{%`t7l1*ve5Vu1H)OtoqE zX}ET7rExX8LJ(gIl#Z&Myo_mdzE&EZ>K?wN!#0GIh{P+Uxyu}U& z#|H&yxXU|y8L_AtEEX`phfh>KW*9Ksj@du|d;&M1INS|Cdp^d4!g5;~jEFI#_p1vG zLX-C+Im4mZ26d)1NfEgYQlQuE<2X=Lsxp4dB$PqY5rAp0%PgJ z8|SU*mJOe&61HwszPCm{kmj8tT^0uHkVAyh*njU#Q6^EE)u7UQqX4-NcTiH1AR|2d zD4=pTvuU#}L)n3cIl@Oxt$tvga4RH~WW9i$7vG;$JJ*hy0%V@RMRcR?v82kCDzPI!?FN4@srI7$?w3o}VBTR;QAN;2=5bc7UE3M*6v>vxs&38y;s_;}>i{HHwMwTF}SOuvYOgaJrHh^nuLM zV_zK(*ZDd|Iueg1NDA#yu}Qr4lw;UQm(`-*Qq}!D!2{X-%9We31??DSfi?ptBIDKY{}0x?$0G4>9~&C#|>2IvRziK z&W)FXNEVw8@ghsEb#4~HnumD88kc9!E2VfeAPBna=@;#TP z2ii8DFu|Y&o0xa=ppCtWhY;bD44;->V_xs)i#CrD-v3dEu7qwS z4qnW3uZg!VrEatz-XFJi6-{}^Jc{FZyiZQoWAOqE<;W9G3;iI?R*Y#A&&}}~76J=B zHX##4x6`!^7M#-O9R`j}JObtzD}1^Im5LXN4?LdCC-mQ>k%N%aZVA=0UKr9>y?#gj zK>YN<$SZM7hacs*w0IcuPw>I^wVWBdxj?MW#;S`R>A-~%~&hW))EqsZp zr~Y4gsN?6;i~axNvH!tp|9?zpZGivM>$jx71u)-TgG2m5>HoSY2FeWn4|4rKSn$?u9k52^%jz zFIrFW;Jo6#r#ad=OIEt^HB;xaTupA}YBjtSQHR!4y?Taw=k18WLGSaV|LEZK_BT#N z8(!799|8mFFZLuSEyZr9`Z1zE=rIOBJNn;UY+Tc>z zsUvl(4QMGQEU)J5Zo2-7hO2CiH?#^p77)iBO)|Y-iOx23**Nmk z(mvubQ6iMS>poC_emUM5aGli4=sdDjN93q^B(3NcPe1hFn~G!j&7NPa?}|SQuV*4S zo%pP$WRpwHmKyF4zoc5YjGMnwNj5ss=qcw{wU$`-AHTO)DOybT%JVT3K<%}gdWyEv z(!h|Y89=xNS$+tP4y94=4(O<8YdCB8783I@^wxd{_ddHaOhag*?S~Q>1uMo%zg{DE zpogTQIk;2XJ7l33_#55I0iUMrSE#;;I|t#e+%s@p$A@o|%w`pb>7D4X_lhc=XiO*n zKq}B~Nb-3WZg|gMUj`mNf<5a}7w>ScHiUCd*KklnU(i^DujbfX729Ik)M(_$5w(Pg z0k3ki_}qV13rV_HS!!3n*LcW|UK@zk2suEys`F=>4Fn`7@bXyUBeQ$;xf-X-y&^aL=p{p#=oi<+V^IuX}V zJu5%~UPiz3`XZO_{Xixu^{yTwfgC$BiKEw-AxSi(l_{K!x&)zRA#N@)vwhVO(h++< z!=6kV#JBUndeo1JlQADasg5WC@JNYxLI+4*a;ntCVN`1NlRKAh@K?h_U3=QKJk}Q^ zWGHLiN<$1jxe-bgU0mGqug2WX39i5Sc z#3;h@(E6XfNKKq6blJq7ysiYO|<6LKT|KN*XOf z253gs5l@JMpXFm|lj<$4x6C*XOwviWh;YRKs%7v_Vb7jiV>gL4No z6gqF##Y{5@joZ#L*~lK=E8s3so=R@_Cg2fs*1b@GLNja(W2ztgZprnX*+9N;sB_to z$;uTced~pg$|~k!Q-BA5a{RK1Brlk;nNg_8>F=u9aLU+ld-PVvKyZvJENM8i^vIfC zR>6%nT~j5HXqn0)RrQO{`FgK4u;IhkP}U(vQm%(T_8nEFntmRV^{3E#eG9B^f8(t} zO>bB~mHAcs4pKGw%}dTp%3L3FLSaJL+TFs|6`fbL@AAVb$2A17obAtOP85}38&_ME zykmyHc3Z@rOB(<$icL;Efhj#*PVwlzo!$}es1j>c z5O5EFK_Dj6v4#nghCznE;V&thLNfK_n>j>}pVu?(HGQyw)4F&Moti-i9XPH+Rk=76k0_rjS8D3hOL^>17ozj0yyCb8S5%*CGWoLL zf_d)tlvqI0Pw&K}0f+4jOU}&EV@fay{My57;x)!eQ11Ly;jpSfL{;2=`B>wxYb|uo0 zXILN~CLosv$)QYPOCTucy5OX)LN2H#rRpWM+Zp-*Lb$c2loN9PthC>m9kKSLF@c>-xl-pLMZL;7s*#4L zSqAkS$rP zwX3Mq82fO--{wms8Xdi&t`kqc=mA3MjgEJzLD99sYG}^EfYK!br44go->lJj0g9rp zkEnX|Z5yEch-{x*1BifZ!a^@YqP5ToEDH#YHZC(Ho8zTFIHGe+X7qKE}c4;Fi8H_U<|+P zX^!kvFt~W=%RT;*Zif|tXV!}09nfdo-edC15r8`6uw*s))`WK*TX;u<4UILlrsi$1 z^w1Yp){p`Y+)fa-Z}9tNn%)KEB`kXFk;^OL+M>)ySfAy7!TBa?Flc)mx@UmU6J5*< z_}Z73!l-`_I)h42PRX6KYrNGns8`H zURry`hG)1YyWp_U6B`zF{$xWC~c_O_^PM>%-;w_sOrXJ!nUYzKClUnH8OvbA~qkESc%w#?Nj}D%)a2bs{tK0KL z)7S4h^NUr7z!><0jD>|`oU_=+0 zsor_Xl@$>mCrfUQ&#>#d2m&O-lX^~&B_>+!T#6^}KE|a%4HtW?Nx!HAc}O8Vn&Wdv zxAmOu@8WMfeHP*Ae<3v?c9Jyycsm9}vT-x^!TP5Ks#mw;Ll|@pvurZc@*m#g9j$L5bQWTxk4~CAE6x zPgWmpsT36#o=9XZSwxPOZKxfUCCh6K%QA9u+diDt_~zg`A{!~zx1}<;Qb~5p0xDkXqc9=tyBp18EZaY=wp7r)>>C4IyTTSRT8~OoXbA&BD8Ry4+-t zYAJa1nNs^c_zQr1U{WT0Y36B{#Br$Gi=)feuwJ|;E(HvgK1Z5=K%xPjLx1-L zt&knaai%+RRvkI7(vViwQac?q3W8+(>xVr}sls*?AMJ}>N$YBZ=;TyEw3 zbpwJl_6$`MBcRT8WU(QaP;;+<$WlL{D+$mRBu0S@&qIYmUg@Fvdz?b{GrZ!9Kg(;4 z%N{Es^BaPzPFo1bdnR=b;;7PXL`%-@XWgQVpt;ZGu-me~cTU2W^WeN~h za=}&_`?S5$HDJICH3teFgeGV!weM0y3*u%d$I8Xx!0>=pEsdZ;mXj!**%79sRWZg<+xKznfTJbi){d;6(qACUZ@gXu^ZGJnF(s>a zpb>%q8ecoL6ANlcTrQBnfh7yKXyl6I{=(^Ef&XZ-46mEjYMpC2#}vW;pl!u}(Y8Ey zbg*w!zM&kATyG^)^dIM5^j4l{5+Q)T_I^pr@1HXZM4z0Fjtu{S@jhD~mofP4AE|p2>icC!R-1e}3-%bz6U{g{`jEe24YcUp9uj`s)o6Xr#51jp%-NK{s$>@F8qQia>^gE@n(tNa9wM$yAaz4r1b10!7fAO+iV z6PMkIc}+3rUIjQ?NyhoQ_NBG=j&5(Bhi!~-dK`Der3rB>+0>DXYX^83(dm6G<~92! zlflq)=_ylC3V`?7_+n@Szd31$HKiV~ey7$}qFwJbT;THgG0UWEas^A+@^kG=w}&yj z1yR6O=beOfqvq8^hOq@Mv9r&_4y|6N2^-uijn^BkNc#766m|vlM+0`63@w%$U-lg@ zPfyZ}72E~YtAlDc47T1>TrYk0DQx2Kd8``96Dn)a{*&>$cS6!b>%FnAAlxK8pML}v zKOc%1zdm@mUlHM5jNC`D<6C6m#O>8Kar`ES_FJMPfPy7IaVHr3zAz{8L{-FvE!|12 zQ)$UyNK6HAvz6BU?a}<)@e!Y$Hh`)(H4cPpK6x$2zuxo|#r?0BATrYzEoiqkvNRR! z9>cYR%SK7vgV%)^n6jtd(M9X^#TYJU<3H1M`F0auBf_l%P`FXn>ho}zj;2S+EUtxX zhm^~0BM|QtVdwS!PiRb74>p?k41x{2$fg8~3Kp+K^B@HypxJ5iSS zfLv&lrtY%ENu+2h#8j9WUYuaRGQZSBmEO>@$WFWYN;8Ybh$zm{s>Wc9TM@ z7j~jHagu#l2R~7NjNIC%3r^gS{YFk3`U(=ybJXlH`|dLz)g1I$h*u%EYNs*f7GfK^ zE|d@Q?R;!1WDK3z&VL)Z!jW$Wef$YHNtKG#WfkFcapol9D z=26RKPs~G&`i%Gx+3ohG$b93n4oDKEz?S_Z<;K%&TErJ4$!@1b))d!AdCH5UgOzY4 z?3&J&MuWd<{JwhKc`+ZG^{~5StVti^qY3V6S^Ie?Q-_{cjl=JZpz*t;Vr#;K1xoV^ zK@2VQUE9lBb@N8edtr74BoKVwZLJ6G;NiXi=9a@*8uDN6lyU?Bpq~~sC*KP|!L0O` zqSSq%T4GWeVNfEAh_<0B$Q?RudRB40X^slUMQusPTpgAlp}eKRh#J|e&m@A7N7V;r zNdFC&(r4eC#%_yrBf1Me-_rIIiH&k%2a?(SPZ)o8iKaBW9rd(E`nn6|bTzS+z67oL z`YT?SnQ6|myd$A-0?#cxyTWbH#j5cU^%fNBjX37VVs1fYdW0CZZ2W>Jgr$g;mUzQA zSHMjZ+rhle;rZ^7@$M%Ijk*~}g(C~!V~Y)wO_fKvhH{6od7@D7I!8)@XmifZYYd5v z&W8xG_2Rz+A$>M=GgkF=&cbWT=Sex1=MP03V>>O91d|&<#-dyiSy3_aS(DChMGmd> z8Cz`h)vuwCT=dGsiVfoZG8Ht@fD9f)5Q)U!mH1*r0?SftDh0q>r9mmNjGEu4kIgr;M5bRg0Y-6Z=Wjp0!I{w!6f* z?S9AC#=-ldt3%~$XEDUAeA}r8xNmC{=s^;Z<>WSR@C?J?DxW@)$kaB?#*xLrRU78D8OSEF zwG`u&^GpJsN0c+NZN;GgR>LyV-d3bfGK(Ya&GX4omv}O0!8gRF?JlEAQQEkT#Ne{b zCQ@!WPOz1?VW;JyG>Fda*Tv{jQ)_gabMy|4nLCmPg*w6Kuu zp=d4!8!+0}rZ4w9opDt(Gm`(A0wbABoShLNBc%{?S8m1op@9l*#iUx)5-5%lc}l({ zKBFQClH0ukQQ0&pfL3l4iq8PFB1-n1IAY|zQBuMka)%$AP~AMAy@S2;6!(85Vqz*s zxpEdLJC7qWOAVa*`Ny&tu)kemPNNf=vVD%Q+00@vF@*N~Wd-rwea`m{3UtS7zE^8w zN`kX|EGq+l2GxGDQM}RQrud~8Gm)udu8B@=i6yYr)m!t6_`S7*#V;q5p!5;ra5%7^ zjqJ~2M?ipCA)i|Ty2)R3DdiC92%5|Xu1BU+Q1=RWwa7Lns);pRdB6ARhQUf_<-?;Y zslo%wQt^%!C))QkIg^}UHtLJ3iogq=^A|=gWuCLvvqEXk!*Vs>XCM(`?XO*Zkf9%Z9>nAZZKvCfXTICfpxm}Kb!NU(HJ`o7tku8U#Lz;mm{ zPi?zT2ORxrI?+Y6p4gg0o@VkRg3z8XTKCFQhu(3}P=GQ@v3*!kXOwk3~a-mr_ zt#1oLOw zH*j?-kSVZh$3Q*E``QE%aTj0RKLjA2OoyQFMvvj%u)5B1hoXsrmzy$8&En(2fr^2l z`$qfL_+#D6sx7Bh7vC-}JI^=X4M2s2bLqM-6IzRw&_sGi{!18TOoA|C4JttjkIG<@ zI8wFaL%d{?SdtczlSvgH;|z0A7kM(DO_{E0M6$`fUnA}hxO(k5VfenJ)mY8s^0{Cu z;4H!SEMep9EOaM#0xdsc27F>X?2|)1a-)@uisF~I@m1P+>-5u>`}vp3f+Le>$u%w5 zv5wssK`N5{0&JEg z<>g9EOCl|{)A&*vPMMC~v?vRzV3(t81r%_%q&8iie5UfLft$BYv^u$CpD$lVzz>~# zclSkJw5hK6B6aToLIy66>3rW+-CrAt49nKlK)F~)QI5$DydKJ*b;Sx6>Az1kW;keA z`n*$eD7?f&2h_oJ`grra}m3@WX$4w7voR!f&KkuI`*kgj!Z zumpkw&IElA1UFVAuT9%amo|(}kVo7`?;#u!FERN?<=vvIJSDAUx;VsO-Ze#G;F=d8 zIO34!1CLyKU;|9h#{IM^>qMz^CJETl&lYRya-q(k370q>+!!no^fsv(d>=0J@-5wH zpb@P5x~OC)1aYmce?7Ju72#!u)t<@1iA(0F5#zHfkryqho?9mvtLd$2{DPfd6h;mn z&AT}KdeMx;io6M1XJ0w5k+t7}GMTB@jEK%2clqsbs}}i)=l+A=#I_wGrgvMbrjoom z0Q40LWBk53&4c0>8rp^~V^6b1aE9jY%JBV>AK&YJ$N@+k21v?75bJ_}x;-40)l*lc z+3~%;*r{$m{Yc{V^&iM-9l3ome|`1KrWTh9quhI@CEWAyC})={xQc8iUtV6jQ9~3N zSSHG)+rWnmEG)b^{HGVc zE|(%DFYRrPJ?|j~QHq#_AzsLLhSA+1em&6xi=hJXB;6bLBz7Jmw91)bGftT%Z53{`8O9$kc}G$ZRt1@Q)GV zxIHv6WS5$oO2+6g#Hx&pq}cWBlnlo&1q_R2g1)@Bslo9wKg~iqoZQ*>#*LUVJ*(C8 zH8N8Fk4$Cc`CSH|T@yD${^>;AGu=@Z`Qh*XCUGKJnA!LE7y6jiUmmj?MuPW}=p)?w zAeR#+V&?Hqt$iHR)ZEIhVpl!uw`_Ie$M9WDSP|Jo==UuLUnFJUa}zIPNAtur)47tP zMAuTC3U8B$mLu21c61ZMj;AcrF;FhgDV?Pu=29f45VK|HdvRX>?EDOOOHEoH3<40Q z_tv>h9%r*q`&Za1e++(9_3@_A@gxNQ+51C-@`5W)%bGIs zXi>_9<8@QY5t*AsAs%^uH9m|G9=+V{ai6M-cIcV{JBkLD+#G3IO$f7DNXsc?rcDUc zKS;n;8t)tM6%i_!0D0&{izf@%zS}JJ*dsFtG&q(&3A z!fUB|bSTbzzMB*p2yMX`B>S&H)PSD$ zczhc^30kr)sl=d0M2t$B@Viv89gS0{PFwAsYwo*%vH*5jpPr6kC1&zpF zU9+YOYo-V@!3D$(*MiDHbnd;v)KHrr3U2YWO=*Jv!?dI=PSoYzuXq>-!)4;fdnB-7 zQf;-|yyDFT5)k{)D>JYfICd~t(VmL_8Sz#I@nIRC;&Y5XEEe>e23EC)@V*u1O_cC4 z`iztpC_4T)F2CQw3po%m&|5Jk)U~VH73lQ1O#g?^Jv7I8 zekX(Ih%?NfI~uA5j8Tn$(hFr<&$KUXG-wY;S1dQ&DRf zf1h{e)fa*JReYY@pQHRiTP)zpRKg4avn{fYtWp(i^UDlAINT{dOnKsrs#7)`B#>$W zV=Yr`)IEd|>+n?uO#fAHU!UZ;-1hbe?g(n3|3!)aECvq^V`Jz@N_v<>N_))B%$fWF zB*9WFrz?W~_Vb6RG#Me>8$iTMzQH*H$_2qe=A+&2uw|5^d=1r8S6#W0C&T9EgeYOQ zLBP~00wj?Q_0kZ4bkIH|fB&G$G1GKfXz+Y|Ji>h>kN@4!4{}?acHNRsr~m~*A56iA zId#20SWi4S{J_=I=O0<{u$_LjLz+xw6+bBg2H*`DB2)4nu^!$(f+uO^s+Ns}ti>R? zlSmgScN~6_+;P}OKZTpueof7)74GfI+qnGcYiJB=3oaMJ|GLW=DX^ihFR`kWKK?-( zpkGg++ikqJf;7hW-ulfqr$_r8PtYi>OSdSClOryw!)@2TlhVX?afkO4f{Da+QC5A} ztoAG4rxojA%Dmc+m#E99BO25day`XIJn60AjW zCJB#cg?rYNFo>Hr8?G^5J>*5;1|CE)D?9u)Cg#OVmBwX@Xx|*J%E0)qP(7|OmZM)* z*#9eyS=2(RrL;1tNs2o3QQs7MD>h@!_k9LBxifCj!+ril!q`}1F?ZUopsXytMeu5BkxxaI?g)YX5`*8g9mkq7$O*lvSh8eL8K3qCZTE zq8;OtMtv!jg?;^SsB7pSXa3we;e1r9Oa;3{O}83Zv8AT1-pD6WVVCrs!pdpWY2s|i zh0HfDGX&>^bEqv)iFc|^jHG8d)pPba4gDeQ!7jtecPPL;sItOsGCpB!iI6fNvBmBa zCpcQ%g|dNyf$`6i7V}T;J3m1227-yZ!C}j(L$}GHjZz>dmT748vmc0dZb1J2 zz4mXh<~gwU3SGCzGKCaL!9v!6hvB-pN{8y}-15|Z!?L&CxnV=^)FigzSQlcPS4v;F zM21&pjZREzg+uQyiQ1=z!3nUq(;MD2==JZqHVq0j2H9@6m{z_-kGQfuFl~_?^a44c z+*F%sDP!Y@p2npH<%Z|U5SpG&CjEMg5)VNeX3As^pGQ-&a~ZW2-&o!?p?1Xykd&;7 zKr9U18Sk))>|MNIORS4BL)3G=Mefg>BiGxpqX9X*(W2!?HwM2E@rk9HXaES7qvYNs z-sw{g=_7^7-+tBjL$DYTOr!#PXQ{x)LW=(C!FM+nP~{)YY{=L1N@y#B>G5yE+E@Z{9paG{))tQrGXb2vG^MDbPKUW!iQumnvN-NO2g6&djhO$RH^yRUr_ zyr;|AbQhsqYd@$a8u=kek32mj=qZU5eX3|M?pTgKjm7g`H-0*%k3R5gkQeSeuT7I7 z_o3x&7q+H#Ny1ptk!bJ=8y!4PQSrlhhVy8FvurLf87)wgbOROK^_eG^uEL;?s)HN# z$QEB+Z&!$m48o3dwzK4yLROph_VD{2? zo=*0axUxDklL4NnwWiAz*rfz^L_+goyV1_&;9_o~p($B)aA}WrkD6N?x4aa3Qg=U~ z7J2u$S27I5??7Lh>Q4D7uU7^=izTrp-B{9z&BfzpgHp!ALA4jVUXZz>n-=yi@0=Ad3<)TmOTD&!*JCs#vN5mi^b{6gjRkki0#_M1)4Ib*y;U zOXX^UOe$%W-Q8cNjthlgt)2@UJq{?U+qZky8Ct>U*?ih>`Du} z#a;U!nnWK@+nEfM`E-jElFY4ej@2E{R3Xng$Tm*((+;yS!}wzP=O@0G-Izu@w|0#U zg~5|~U+&0*PU~-{W`rjkIr!?wy8|8VPf)YM1zx`_NW{Z?xd~7~F607MZsdVT@x(aJ zKvg5Eai+K^N!|874o1VxAfhza70`N6KygGxlZg{y#kL}2Xy~xcZUHwcaCj3y<&)G{ z>OCyn!3VcOif3%LqE)w@Gy67Oo9B}GU{|nwoI~co)8=pc@5yD(n!7V*?Aw!t+ko)t zT9q~9%gePQ3D5nQA~K&vW-IlEje$M)#T~4XOXr-Ya?a)3wUiGNANG34w{_*gK=Gf` z*+Dky=ldC7#7;H9$K3w=CSL-=sM7*6ioFw`xF$}**h1V{%ZRMiV+g*AyHXc$CCl}N zV1i?@Uk&z6fAJWA^m$r{j@>sbQUtLab*yZX%BfYT}CMOlm~b z2FPNuT+^EOP18))gJ8p#BPtZ;=D-N?rG2X%z*RH)$k)q6in_OwTxOC~m9$hY3oSM(NA&I&D-g z&fFA&pNW=&DrMTj5hA$desnu3$X(+iPWdSQZdEM$)j0=!GA918F{0g7JC3uq04~~F zv4LhD)#AKMPe!%&X6a%3!&t0ciuxU%!SciQf5Oho6D{L&HB7uvl!^Hfl(#T`xzdo> z5-xOh;V0q+;8D6OGQq@*OTerxZb|s8s{qd&9r>=R^&@8JvoHV5j5k;n@#3m-)%GfG zjoPyHxB{b+E^ZWiJDP9w&Sn$WD441QMBK%^Arq^ZZ}h&d*(b~@WjeQkb!XRe=n=;e(D*#c~tm?>`5C=Ci8MI zw&=hSvOn20dha)qy{G*#nF_Tsz3oGWzA3-5^_z2t_;k9H`3t&~Y||5mUvpa@wzK>h zZ}llzr>A5IaW5dzM}WfApwcsyGrtd`e1b`b$t~;nWJztqUIl}rPuiiPklCz5xs~{Z z6(Kn*V_{_|O(a?X!!T_&@rptz*_RsArid^js^YMAkLBT*ZK<>)1rPtY?zh8-|CQfw4CF{%h zFb888^Vy1F$qVVsf*-mPl%gjKUY?qb|!Wn%RbA? z;GC4ziyzaaYG3>d7`XQA7bOR8tL&<*EKv798@_D>bAQ^6U(nG&zueJnpLG8zF^MmR zz5nwf4^2?%OVXp``;}r?c#}b2XAi6R{G+>KPAA`nr|dhgt51unmM(s*{GlN7EF7>< zQK}QZO()r^0Ql@oY<4gr9kZGqh$~uz>1tU(i9!YY%?RtDC7;kl>0=Bo-@(I`#RDix za=YXn9D3Emo08@HUmDv0DWnX`qpCTIsz8CADdvCBww9zobm*U8yj8v0Zo`S2JTy?E zu#*#6foL2Od^`_G3i!XUK7r_XsEbxtx!Z{1f6}M_*55z1m&>$x6b23aVH=GP&L;z0 zE+VeaHDOIP)c>0J55UH8NQ>u!v&iO`C-{O|%Gfo#Tz3JfayIb3iJf@k%k|5mq9WU- zk^y{Lvf8zIU)FvIe<%-Rv!V2v0x!wGQP9;g#-STfG!GHLFY1n>3Sye$I~%3ku&LW` zYuXEKpY6u6oQWB=?xUVx?@6(N?#2p;5vuv-(o&J=oG(FtBk`A;FaF^=IBpIL1DaCc zpPSB|kH)Kxjmet+f-+XQt-djjuGUQ!x2nPkVD*TJiop z*!1T(LErO>+pl*yRu%Y zs!30?z5TnLUo{*`Vznhu8y_IUac2rBhu@B3&sGDShaT_i=2}PZn>ie){2i_1Yxh4& z`qu>C4PO$iLf7~?+5Mkg?>e}XNRnP82?XCII=DGx<3JQ!#H~RSYUm1^h2BOGxkEv+ zWdh);HzJmKZJDsoF8Q-V-%S1Gv$C3?&DG4*WjbkX!c1JTEAfnUP;&>>Y$+b1dmA9} z>M;B4y{9DW+zxx5KuALcQMepU7{EJVSkcM=*_hG$%D&^fd|CHr|E9gxa>$d8>%jIjTM!}E#ZYJUdnA~|fEu2kD^pWT>iAS*c zEAZ{i48RWe?dS5mnXaa?rf#fgyoxPD1m0HA@aVg2Q#XB)dwv3dg3D^w&7ft$mZ9rmiZ@JE5XM!fkwvf++mtVF1W--Vn9VhVqz7Vcbb|EH9IWt7`D>unwF z;+02Ejj|jiQ5`@SGdp##N1YY(-U25ykTCOO24@^;|LtYWxwbYN`U#{mYUs9wiIrrlxFkq$u2ygKzI|Ks(X>p88IxQnl?}#;jIFGPBg4i0@nB1qL! zThU+2j0#A!eT>@#)&7|-xc$UijRvQ%`>I_pEb^wG^2;L4WgrPNH&YBuY8XCr?q_x~pcV!CP391FW!W@#Orn+(mYyOJ5omSlM#ab)=0;n- zyHJm$%}Vi_l8vL-G2zeueL$s1#Hs~?e+v(~zlEPakht8%6{9i#EiMhS^IGzt6iIaW zt3Jy$gH%>*O<-4UKYP8PfhrV{fzIAP?pSSV7%M8!a)(#YS;lU_)Br-TE4ervvPf*Hlm~-+ntSQfy**quNv;|m z%wHF`3v}RWSqKZsN-uXkTAG#Jw>@UI)4k-6VtI}AyqC@gM;+`Mm($0WgqSvKD4Gh8 zJ0XL5%F1VK>||&8fY6j&l#lrFVXNLwH!!m}*PgHVS{1%qcpjoWqr3VI*zyaHE8n(Z zB_1<)wK+K6KA8bjY>fWRs_c$Bw_V}30n%XNKC<=Wk+>20eX{|NkOd^ECUc}Z3PsEt zCK7O4fiync?CB#|vl59+J2&$gWgrSiAijP9nW|FePFjR3Wcjhs0PtcJQVLR*9BEVD zV3*Z}zd67RvOY9yg2aQ%Dbnl^qM@glrpS1gEGvXysfq7nsygS2 z(_44cbhbNn&!jTO>uSPCGkt!mw6*bB$>WQ;Y$ia6=@&yjB3zvq$@MPDgwo=7d7!LGwN+yhB5ZGX-9-}U%dVRmVc z#hy@A@2NT?lI35Ti`D7&)+35AG9YM%*#D@ZDKV&&+m)V&vW=_vnH=N zF)02C19dT6L6VVR2$%c+VeLJjn#|Td-ZSG2SWr|P2wemzfkX{Ap9i(?qfgoK7HGn8px|C2tyE`~@&i9>r*SdGzd)Hzu6;s~*zPmlo|M~xR zm(q&fjGFgQ%xIe@p^tg;$?Z*-gvMc0pqmGnOh3b^0QnakkzM`cSNbl0Zgc%6sGsEs z+5)bz<-w*@DR6}~Y%9iDYWPE`$N3kepRH) zb>|=|=ExMZ1!3V>M`lW{Dj>>>D9!o%s1I{M(O3o3+ET~oKd!A zR89mwgs8CeKvg-8pE## zQ@ALQBfS02N6vpXQ=`%AsC6tI!bdS?!ogIRmiFU{q}x$WpeJ z|Hwf3O8AmtZMwmd)=}gC6TO(U?d~)By9ZmaS!dV`J2oEojG)9?44G#d=8o+;Um}sq zJCmhni`o%vVo{0GNo^&HZ$90m?_#nUr-?S)iCHp3bVKM`T^|XYymu{+BFzi^zwaAb z$s)8SO%8_bj!xA|9RpUf3QKDJolW}#W&=5ttHs|~n-AwAb``}=ktjbki{n;2u?!LE zJI}h>I?DvIRy`F~rT;b2^o^Em`X7=}Rj({%ycDg@f z8?tqQ`QV}3LY&^U{?Xgh`>kguA0~Vg!=l931QHxBf-=-C!nPU`R@WwU69=i3Cv5__-VBM4_F_{ALsajj)Lbl1Gs0o8Wt%%Sm1iLm!+Z9AEc9aKQlC zV|AyAEK%lLGO&H`d#?3p&OxdtS&`btTXT-cS%=<2pbg3#GonZ;sU#5delmS(xJ4qT z<DRocMU6}?Hn z0m|{u--KHH6o#K!nXw(t&St4L_afyT4U}$g%(1FEo+cQQn*C=z_wuy7Io6zA_9hi= z+rP6HSe~2CdP9v-Sac`25gvE$+aK<}9rLsN=Xreoc}3|-kS<7#Vj;%Ylg_aZnHE05 zX>QX5oQXo7yc-ma(JAHh%jdMBu78K4^f+sv`~szU4s0F&4w~0}=gISQj(kr)+tsRx z=inFS)PNZf4==w~&cwLW96&0#B~4U+$H)9p6J|iKt)37CKFO;fY9e_ESBgpD9<4u! zR-TePj-!F95;mX~_&?v$ci>h7EQdc0gg;-N^p80EUjh7i=!%!Wc62SL=79JOdeAR% zgP8}~qS7|hE6P-V)!v1EbRJR6m8(E|Q?iSAhyA|P7CE!;(-Q;O074qUGp;7=*yfdM zYHP(9395a5zM-lZ6$&qLZqplJ`TomI<~z{d1}PAYyP0+_y^S>1uRS>^L&jkZi%2WB z-;0;;x|AKs7BWe}FUL<|`tf|OVXfZM=~wJZ!P#*ie89>k4n5!Y#n;_!=ul5jZ`#xF z4W%%cc|u^e6GFg3+6Dd?9V zTd}I;&Fy$<`gpeJTdp3#QgnsuC^{$WYElv_AL>^SciS*i2xzY8O=7aJud1L7o&=Sq zm5>^w#`htPw?@RbRV#8{jMtHS8avc1Jh@@!WU&ep(5rWtm%m4X;`LX*RyXqm2>8p~ zHrMw}2Nls)DP4a*eEfL1xm-z(P_(!&R0@g?k$3{xZ1vDs8RhPl)`#uuu} zkH4`{<*z4uRG0Vt)vpK^mm_H_wn3{TaJeQHi~$`9tNx6;)mb=Cd*|P1raZmqtV=M- zufwKV{FYpOuOko>Qf$F}QYy01}?OIuw5+$!%eyXE_;xuW4W?CK8U+~zt}YUHPK1=@IcK|HQ74Rfo=0OHuE>vy>6!f&ORJRL5YD8S82k&-`NgF(0ujqYwy;$zDbyhMb>=K~rqNZ9%LHkLdUpPzjWEYn(CU3@B74rvk4eZ5)qq>!T zx9iHspg-Z1>KkTp-Mukv=IR>_aW8DFEPmj(3z`iW@|$YPZDE=kic*r52b>Wy@QC-R z9nYc{HJ6N!vUhk<(VwnBiwn-|2*S1+@YsuW$SkeDdicVM>MPo>HHFqv~iBoyQO^JeUJhl znmjD(t-6`v!r0h>#`~INvfDriDN_|$h0&BleY<-B(_^ire^$Y*w zY!=MZB(%^2+cJW05?)z)l=maQ`rX0a&+JKH?2`3JSkF;uc%jU*B(M!iU7?*@_R;uT zS0V6)`<;&Z=}c#DeaiA)=DI+?_OgyCDJYK(pH&!Z7^KzgCf&C8nKbItq?MW@&n=;p z#&IHWV5IN!bkJPedGafTO1#xSS|()bh-1?~k0fU%(wxhg&=}{99bN|DrD6wFS3cjwRUEEl~$b@H#U8WC*ef)VO@A5)m#J*0)+x798k8g%b&|1FLCo^vkjS5rdd|*|T4c=9x=?R~co_w2{q6yRLo@C1jckA4W zy%fIA6JanWQQb#1*WQ%h?7Je0S-wu113Bf)RUu;(>tVKTSFg)8&M`#X!IQUPy{PX+ zMKvwN8^%w{-$7GYQS97c-e#amqixL%5y-uTlWMWKocu8*+dbZPB-aM=7JePZ-qve0 z5m|Z&W4+&X+npQRI!8tRN@c>lON-c zU5(@qUB>^GRpmDdLZX>Co7u$-v=?cmNXxNM^?XmdbBWvPyv#3PEaf~8gtNRf8z3ZY zoVr{BBXh|AJ6e&J%reDNKIFk`h5xg-df~=Ve4N3nmPfXB>bCBQdbe=5lX80HWuj$B&Z;6X@5?dCr6cTMs-*Ag8fh2r1N)5u zP49P>l$*-pMj2CYV!Fxw_xKm;C}Q6Jqc3sVBiVDpO=Izp2<8dmO& zCQd4v{{C-2VQrZ}XPIhnTeX8IuvKc%-DZ04eg6_9Xgh z(+pJ9xbESL3*qdE&l2fqX&-szOgw_{cHdC(%XLtuQs>2@8$@_J%N2&zB;qbGB(U>c zhOpAAzEvfT#pWrc-xe?rZRl>%OaK%y+6yle7Bs9K$X#D2HPzS)o{zg!BUEqC`KY_g zdXxD(YDlVv%kS`WKU1m-rQL4}ZK7A`1;wvO865F*ukF3VY2Vz@1fq3rF|V{A^N!N%2-91rzj$~RQQhur=As1*K)e1EwO}Ww zzm7TAC41cCUL;!U6WIq<34wc#@nO<;mF(F{7Oul-C5o7TcI_$RdvVgva6l9Pa)nrk zN}S+i#!m!lSz1tPaWaqoqfdrJwZ%pul0|cHB6n244z>|Y`EfF4)+i;gcm}1gcr(ry|W{an@)IMF^upZ*6Y>`S8Aw?mjzSoLU zz0oe-He{U{*~_&;^!AUx2e$X#_uoaW)q*{w$KB+b7JhqyB%pLrQ))748oP^Fh^LqY zXi^V(?R|CdR@ah@rTn)HZSQSas8<9}hI$8UYkka8)IYRe#J<+Gf!yU=&8v-tCcr@*;IE;@vZ{VhCA zoPk9mv69>xcfp7~`=36a?|{Swrk(ytk<|n%R#CCc!iTOk#aql)R9Grk_)Ev_d=_NB z7jXS`@*;Z-TFuK(9`-WlP&2*9SO_G|_Bs7D>n%M+R5ksW*HXh>(z+41tM4DRB{;N9 zmOfWjoC$d&U+^1i7ZSD8i2#-S#%_lTVx2bEh0ZsG2D*CBJ=l&)KxkncW>wJ3Ym5|Gw3f?zFdhe>-`| zfa6p5xv}uPKct{bnjZl}-a}$l#C(G6eX|C$dWEg+_}Xg<#b@^5Slc0tI_m zcRO}0!u_#;dG;0?GL@3RQA6dnPEuECP&)qD6j0A4|DabxB*2cGu!846SIvjnrj$Ns zAC!pgK*cLsupJW9=^g7bh~Fp!Y)Fx#2S}1^D#GpjET~C$f9Wohqxg5Pe_duO6gSEe zk@kMieV?-N;C7twSYYY_gjegA>~G)H{BjI5d@0Vy-9+F6Iu2q;S8g*5qA2IJ<(sIj zcgamg;1;fPt0R?PHmHEFD*Txp-{=0`{Lf{=iz=?{=G>p8CM)V~Z(7P~kyWMm#|Ntq zdER86>~hvhO_X~+Zqz;HeAd>?v5xh-T zBf)P^b?Ll_+|y7uH~n~Zz_D(ocwtbjbry*enNePMn|&S__jfV6vi;T2TojX?ITQO$ z%<0A@QW<%H`xkSNLo~ggu_6dh*-;t3LBk}ZUw|)YnzZ4bnC z$}%QWonGiiHYCe~{EzGN)w!x7Zu?7upaIFXAO!ouKIFjLx|`iv=I-^KWpzbtiu8VJ zWK3y#N|!bOFjaGrEW_OW&NE@`SwIDJ&|b3(_7rJ=#Q`jGYmYMQ1?6gn-{XCuh<>A(owayDjG=08(pZf$UH*Z=~l3 zLm{4zd8uySZZ87w8gE(Rqznt4?7J`{RQ$BOY}*b=;G&nsWJR?Bg~wF;aqnvk7kSJ< zg;TTtiFkaE$l?uwE|G<+0-EKg28C-J>zG_&DqB1rxz{@U?pVccPVH=0ZI!TWO6j9d zXk$Kocg?5YHBBqzWY6i$-EG1$VZ@Z2@tHc~o?$D&aO7?RdM~g!B75T2qAgn&-}KQ! zo?e;AiJaSkl3I(I)D$1{>4K>8;l>%Uth94_y~D6ju6f6BB!my|o!7)5dD^@*run7s z)_TUesGgX{+F4y@`aQmIW|`P>&QJVzQsgUvw3lWrd##dF020AAgkkNptWU(TySf9N z&&w|_TCM+5Bq3p;LwMWtufz85L)RokA`iJzV6m!=MdduS3PY}{jS7*MbdPjodD4jr zQjq20FU1mZ8KSZp^iEzqhNEL#?(Ajti^)X6J>w0R1X1)G1&T%IPDZ_T077sZcRI#8 zY15dDA1m{MtY*r`rig3|C9yZBnrpreZZsgV(FiQQ{q;2x9IDvpcV{m1A|nAllVi9h z;|L2LSKV%OHpcvnVBNHc!nO@CSafEHqqSsrt3Z+x@v;vVy>)Itmuq*nC5}h1@uj0=qn}c61V`*bQ+FV; z#;GyXtVI={1~yo{gge_^RPp+J-@038$oUG}@MsG!)suOd6`_Uii4|tv_}f{=@nRUz z@Yl}hiZCN%MOLq_KO~vjcd0h2dxI>ZzeSBdj|qEarIK6|HSB0Xfh)BrZsp+v;LIuG z*c-YG>GKJXM(Vr6Em?^9V@De0Zw4>A@U(GNkA#?=Y7D;=YCY^5F$Rj2_cY#Tvw3wU z(!zTC>_GA~QgrPXO3#M`F#OAi2c428)9a2NMQ#2By?|;Weyjz-nL$h6mG(ZnI!PMe+DBQ`|L@ZCNUN~ra3->;p72bL4hLXHRO6H&mRyoV@@wIIs~_A4l~kQ|umEws9nNd!#d?#Wd?9*tvkG~0 zz4C~kjg6~qJy~CEe0^+swedAG@pH(i2g^R*zHCKo=1`TetQOFK2QugCJLF*;vY>@H zQg%)8(|L!rI>E5X%9EAAjV_w!zOx~+B^AP)Y}u+C^IhY#m^y1%W;x}^s4SNXxfR*V z91M9R_tK}8X*JrRrVKp}_pTDQ3f?G6dx;2;^aNcOP2&uoEGgtGqC$<))r)LS8v1@4xM=OjnJ<%SJP*P_Wq!$LmWj z&FdUNKkXVfX8P8Qgb1@MFBuD72IAxqXw`&s)4^88Mv+9OSz9T-(bd|g!Q zdM6uOJe2)P_og4)D(6gb{!pO9ShjC(th zXFDT6t~K=A3P%?E?^y@bTx;xnu%Y31y3=Q0eSSw)M-cy?4D{H{_b<9VDuA?)x^AYM zWQmBb)P$=}mbSiausjl4nIkz(*XtG~bpfAnvOlFh%0fs}EV2n}CA1xremqaKUqhZQ zAWJ=1q4BUNNRO5p>MZ02C}7qa=V1*q9onQfs{_iaQqC{hz=su zpVeBPk?M)jm|WFCwy>`E!_gHHY#hUG9D22qi)V#3%s`|knO&!k=cx3EzN;n3-p+!+;2US3WRV0{pv#xG&Mp@D zkG~N=J(wFiGi&T&p14S_P|3v;NV2KVg*25mY=p8FF@JfiGsb)rWw%=P;0ds*`z*_i z9i>I|ags0&pv_!1c}Y`ywF50}(j~;hM7zK@pSF8CCAXs@9S*jg-8sBEN&uH<({FQf z***X;3MOt*Ps_(Q9KR1NT1{&SAY@5?DHA>EMkxiPMSKt@nSx(MWRK8PX9PqYh_&vB zjZpgk2ozO5yAu@~=BWo_uZO`Hu z+Z&`GUS=Bp7V4J+k4B#LF9ZT>tW=F*;!D&_i{Dc?63{Nk(PnG`-~O;aGsi!cnBE+* zzGgL%*kKln9Driyfswb|pbDuigiQnz&LOqZTQ36!A79P zBrki$Z$r{izYTNznZ{0lYx*4_{*kJ=G8M_e(CJI;{i*ztbX)|_NAlLgIznCBjO(A( z>wA5N9Y}L@*LzN^V37*&Kiid=VxPU2b$P{eEubVxqq%&K9 z5!iXBWU}x9(^>6uvd$}lWBfr2=`a4F?k(aJ}q0Qe_MH1s5+ z=Ui-E#zbV(hVcSX6&PvJP?dMwhe~S1qi5^UnA5SnGq@SL{sO&%z|A)IyBqc-H4Hioqz86l0hHwRos~%l4!CLsU@GDKjZwbM!BFh(n>BJX zk}X#h+$Nq1nQz_xv>Y-TdToP$?HBqaNYvl=TgRLB@E}p+ZWUrW6*9SyFb_s!!PN?9ttrok4v(sqs zPu2dp`}Qw8@Vl+5M<1S4cPD+4*F1P&LhPt=>8a3_`;Uv6!s?d89U%igAQoGmnePbq zRM@4DVWnkZ{N?kkPdH444N4$JwR&k@x4L;{>;k-YHy7Y@I zQT$m%6gFIKs3_cr+e6up)ApOJPXb$>H2Z_O#-yywxjAj2GOJ)M*IHW3Q_pH5f&F+b zk(YFHyRLacf0#W1^7jQSORK2;qNJn!|p+IycO!-u`Yl zhUNkvS#FY!0lAe75*sPRbNr4WfRl+#QrTR*tMh>p?NqHLt;+s0oL^drSkTsm2aQ>; z@=O-2{YK07uJ_YtN9&U{mS)Rb4v+Pli`l7g8dghnN9nNVV@g-ch$A_E!$sbA7WBD| z3&uw|@z-}OQ5?K2Z?br>+?g#DQp?@PyGhOwbJK7RfO}z@ly6w`{(3dNl(yY@FO>sx zyhSIqK>zU3G}Jg2s~2l|bB5rZBxpRA8^-YGm?AYJADIb-qOGq?I!&NK<-OJS#oyZw zmeb}W)-K!A?z8QyHF?xisyF4$K49$nc-zq9mh41BZQbU@tgBTr-&6<%6r;Cta@J!> ztd!Bq+eJ=_A$PJfl(#FFulIarmmcjzYvA*|m4XS)(hK<#gSSen8y%QYn05QYvY_Kw zIqxXcm+#(Lybt6G_%QWERZ)Dw4+o}QUyPdt;YD}0qcP>MP7QsXqr95ITSPn|&|l|% zZq9U2oQbKx_Xfn(^abh$-HPvU@`$1S9?7L?dQ1n;A6k~b6j8&!cZyesl z%DUZTUcFmv5xZCLB|5iUcjE575Z@>sBYR^JRkR%K7j0y4spR&m!N&%s4DpXWtM@Zo zTvcMfAI9I9RA{JhoYgR>O{4Nod$=e5OLxS#c@`)4X?0m@2`4ZhrmgCqm&WmkqQV~q z4|ziqH_fhZnej{NS~4d7;Rlf{p<3|91&vd#(D@kDTjT&39G&Z0|J|oEQkN_jmtQp*oUK;6VS{WB|QQ z3tjUD4&#vA(}dU8WIpKyKSIbFr!#t+#LrzbphpCj+T(s`pfc?5_Pii<MO+67RT!jf;TIuw>&LP@;Nn+mdB(NZvMD1I;;LkN2(Pfa!f;X+$q z;BWO-ZTIIgj;SCns*F_^9Pw~GE3u&6GJ8GO@g%E)7$2cyLZ17G8LK8a?>7C2*_|(8 z-&Tv53+q09{qh6nHx=8;z4B7?J97R%M&wWDJNRg&E8>UZ*0;>w9eOe*qFd0+xnXWBU*uFMd>S7b$it^`29WP9;qVff ztRSJB`;&R6aVVU3OvxY6;wux0&rrhaOsXMPpC-vYVY#D$t@$=!n?g6vEF zaybe8oRI6Ln{s_?EUTf0vT84uP#9r3`SZa^beBDZk0fLucF<3n>Uj0rSdHx4{6BYL z6=K)B@Ok!;b>Q(WKnm8y6M_e|_0qwJbW2$vBaKK;=y&%dFypKI?9_CU{+_emi}*;% z-^^h3*f~+TrVXl%3tyRL)8w|8oQNzHP5Yc7O|eRzY#?Zh{gGP*IM0t|42OkA?fMmH zIy2U!g32=(hKd-lFLxaLK>u=nGo+7`jWAX|f3&U3Ar>V&?L{Vw3H1@KU~DSX*harx z_BQW*hSEej$RSiS9U{{L=gEzILPf$~V~+1mjB^#=T8K9{z44xHpvMdeo8 z{C(-oMZ5C$RimmUG!w)drhYbb1JN3I@~Wr`ZC(_P&M+568l}5BUfmToKE(;JlMJ2==hu0cPi)n zb*5w?FprBNbyz=FM_az_F?$HHpy7cV&zn%>wnC0aolbyd!26M}IL*TN{YlUm(ySPA zJ}Xk4m+tL`*!KNye+n{+lE9OkW-rc7{Sr(C(ph8GH_fTTSTVwx&ZdWhk^b7G9G3?y zsR#}F*!b>NIcX<#J0SacFKGgxJTlxB^w(Su=J}t$&ZdnwY)2z zSR;!|eNR^9xLe1o5;@M-SW^b??~E}GJ9qN;rpK0){7*~D{7_3(%#WwBSCQ?=9iLV` zzaJ-?W&L1BvDMu=A6-3n*X-`-bJ~jKxz0el)|1gvTp(rXJ$G&24cGKY-xfmsK*{O8 z+Np(G{{`kcD|YfNJ;_Drh)2Igj{aHPUcu7?Hm#U{Z(2by zs`ulJ>N~_uz9Lc@--A=hyaeqowmbZx`-$u~CcYCh(X>&9TVs4>rtxuh<9LavsAFW_ z$@}oH_g13v4B8nB;&(TUFNsO#o#5%j!*6Z(CjE{fH|qExvop)xI~AYK(8# z;~&gu^;VdgZzgd&p;a6t{fm$%c5cU8os|8fb!!zpg(Q?tVC5j~SSlVwWgz)_4kE1# zj+oyD`G1}Z&L@BRbprg-2$p{40J}ok_qr`6S(7-Pq?D8UB&v2L5adp8XBYA|?+&ey zV2nkrM~$uYL1<8En*_i=Mcg1Tr)Vc!Apo-Hf)2ftH!fwRHSzQ?{p@gMCmm_%bI7(v zB5W3UG_M-Fjgvy=6>I-R>y;lgtp^>f^im)M^4P;~j?hz^(szwS0k#*}Eb%1~11_>K zLQCyIb;Hs0rU||DDQfdP5%eQOvUmgFrp8Pqx2PmFniuXn6*#s60K9Xgj}c=q*cUJ9 zfYc3rER2El(M($}Y8~~1@=*W3e{VQIxQeo%^Hn; z$rG<7VhQO5b9?`V$P~h^Z%yRs4V>YNhFnDcW4tMBb3dEcTG@`1pxOdlJ1I*^W*;K5 zB=SRD<^bA~y3`xa3*c!SX`W^$-t>WW06j3)%G`Br-syDFsg%BRtYxoygyf-mu&ki= zn9>I;kfsp=oUaw+{I*5Xwrs7ZZ+gg~_V2!1vS$~Wo&H~AUw!%lX0jyM|2O(oAx)ug zj6V)#zjBtHEQ^$dMi;pwcWHqE5j>s6E@4+_VH7%eiOU+C6NqMzV+{d-%(nchL>5M^ zYwTN=Gu|geFP;R~URJAU*KT^NY_pR}?FltJqe}U++Q!ojBJ@AP%gV>)ymXV3S-w*< zzxCr`wG(Z^IY$f#M1?b_>mZZzdFHrpM!R;}-4J#AkKppAQ{pOl4jiNw7FGXdvG&s8 z;Qu3mAA6p_xBOzmcB%8J*|dc#p2^n#ljczL;~7S6{yy4?%$pJ4@)z8Hx=hNWd35r{~V?{bX``oTBu0P zG9RNc99KCY(MTLPJCqzFi_*gbp4DX0YH4GsH;KvS$UrIj{Vg$W4=7`GOncwuBpZlZ zkwPtkJRsBUYMlnyR>8EhK$Zz*6nkkYt5DY3)0Eh315pV11_0!(CSjd0s002RTblcp zf(S&c`2}XFT??_!rkdVfkZw7dOdKf37r0!?;#tUZBpBFg>#p_GWumY=qNEujgaph@ zQVhh}8@D^d^=|@?5F{HCa!E_yfLFG#OZZ0K8)XTRCE-zIJUDza#@(4mD#Vt1vaNh2 zBeI)B)`X~T-{iod>vaq`=^k~nMWXMdqX=<_a)X>k&jw{i%78*vmeM}fv?xe|1{X;Z zvm`-RCgRdfHkDg;bHpMJZm)UxFL@Mlm)4n`yJkR}nPfFTUhH}s7kovfN8ZjuW9eR= zI0hxvu7ROo@9x*pi&Zw6sL}UdlWa_aWrmu66po+JrdyfQ5{u*9vosoHH>ZA6=bzEi z2I$GJm0{vH#h$`628$JNh-iES+7us$KG)V((2e1+kFVxpj$!$tdS2x`(#!;J$$K>d zbdNBW!Z|fW^`u@|Bb>DJ#<}7 zl+jwYqCn!-4q%(6dVKL30iKTccnH9OcoejnZ)545ich~%o<5o8X9CvI>M-$xuYqH2 zK(h_ePC&M7Z9)`Dt(iC*VIF^X@od$0hM~Qfg%w_E=HIJ^+q_v3{)u~j&EsEkhvq3m zPdZ#NmS1LzEc>gqAS9lv@;bwi1fDmhXj)--_$*g%vV9Pn7++`er^dv9vjwYd{BzTp zmy<$R!DtAF$rbZi;YhM@Vayz_AzU{$@R#ADsb2ZG0EaB|Zfyw(9dg358^0K4o`bnF zTVv8p%q+#+Xq4$qg4g(9`@?8iR2ack)ufs^P7Azr@_A zcK+r4tI$V)sF`Df=2!e*5L@o6y5gPr8finC?3PryZ$9hmPKuy3=nvcH3srVyyh8Q9 zj0g1Vv=V7>QGB_TQE+x^9dvd-R1Jo~he-iK=iOU#pQt<-dR(I1c@!eKVibl;*}mZ; z*o+ocf|b@Yczc1qV(NQO1^)6?53yjUB0UHf08TH;*Ye_T3M%3Sa!L@;359iAWe53Z zkq$c`J@H^~e5ISate6DOb89`DqyD#<8 z&hj=n*AKL|xZeMU16E_f_zC%b#X;fenmI8s$${cX5&@Lth%#E@4Bj%+x>Xqm) z&ohm`k4K*Q5VBHgMyyu14GUKqvLX#uc7&Rah6ybLIQde%-=MWP4gG{Nt4O~QiHmKK z_L0ptWTRL}ytUzk*yEm@Us<%CfK*4#oONCCYMC_NPpfOn8t^+K)&aKP3X<&$Ky(&L z?AH&Bm?958(yZ{G=+S?Te0qSK42mQoQSSjg`;E+06q-oPtJNOcY(!QQ;!z`R1Cn*b zZ)cTfn(4^h+$@|kL@o=WU`~AH!N!5RkM#7ZK7oRR$&3yNc1Lad1&)6EJ$l6B#U06L zj22Bbi7_czRqHSTyI4@Dr7Dth`k;$crxEzNumNn*Rv3HEwj@tG^>aUyzFlgD7Cx)uUcIKcVRFlbeU~ZaXJT>c+<_ta`J)k3lB1=dXzQ`H#vQp6}B}`eu&b4tS(R z79mKsa}boHtS?c}? zk(oo@NBq=DsMxSG?b&GX0jhRj|Br9LH+)V+wX8cFMR`2-ReJC}sjLk51&9lyLLt?@ zW{wdC@M}I});K|Ks@7bb(4J?9?edAu{T8Ml$$VF>yg(ux&gwec>YzYlQgFPA0$KJ8 zS;yYj!~mM|8{l|E=^SDZCi`pEGex+LgdY-3*B-~!>;EQBf@Kb(^jw%4qt12_djmJ$v%u+)Sn&F%Lzy zWHxjvwI)gg={_2XqEu@uIqY$T8ak)Dwt9S+85{+0RF7ceuD4;lyHqnuKl4Lh;~ z>iQOz!(|1dv$K#!QnN!PdLv`{@n5>#0Pe5>_nLfTfU~FTq?N!Mw1DahwIcd z@Qh<>WXkm2Ta;dTm;9a3bqks}n3oJ+vdQy=WloOCUwCkXLPUeyaC24x;Xs7n{fPsGgD92 zF+~R}k)})u&rsGwI)kpp6L1esy6IO!#3{L$jJ)Nyx{>=N6+y(3fM%OG?wm|KatRK@ahmpDW?@DO}Tj$1k3nv`D!g3@_Rv-Bx4^xP+1fkCX#VDK+zfBMd?Uh^=4(Z5^$^A6w zPno*h!86qz+F$6BHskDp$6R(`8ykjG6L_clkABV4!2%?01Y0h-e0qm#1S*e{( z8;Tm}=E-sXDiGZBl`FO8;V30-GOw)WLVMMUxpK&GaD@>;FxY<0Tr}J3l__09#elqD z*v>1+7^HN^#wv;eMQftC-0k&p+Z#pigGG&33p#Zebm@9k|n1Tzh-Gl6X_Xc^Cd=om_hG^?071r7g%DX4ug7xxzu8k zR~%u@u}b<1Kw*nYV!MyJ=YiLDqPnMc@~KxDflx79Q<}?y_E6j~-9-|H{E5=~YvK5Z zsf#-1AxYYz!|+iXh#k3`jmhveG~d#KkuB4A9|e&#bxa_(CCp(8@%NpZ%|IwNd-HdO zRI^dtWIj+&=IHO?AGTtH>O$`?sV7tCL{cv2!fZGw1SAv_Xm6qIl_%Lbk5oip)Xq~d zQ-<4nS1ZcVnE}^TZbjQ_dRp1sT~F(%=gK)odr&j~zH6{VH?U27WYB1%hS-DDK_vRc z2TN^SZkSU*DJ;k6Gbwj-Qt37NW%N2$?}n#WgY!tmqQyUZeKP-7g>eHSFe>S8(gns! zZY(q%MipD|15rJ4pEYCp;d(`!m?xkH5mg?q<7Lccn$dE~=?W;$2+D3OdrYr8{UF1T zBVPF^^Ac-uYO77%?_L#%>IlJSNqkAND{oD{3eXuSEYFVH_I7XmhNo$R?@b6!%KaQ$ ziWbgEp6~y(@~$A4h^5r(7ud-Pae;G_94RWcH!M(GzsF{|cHe6YF1Un#WYw!w&_7N| zTZo`arlr%4A4All%sIJK5}-Fk^$Zm&BW)-@O_}0v7s;VM*LSZEv2!&_h=f3b@yfFVbv9QXrxE?aX7~#Q%*NV?@?p@)LsfTU04I@Sy5Wv?({Iv~x zf0bq^(G41nGN1XojhCrlip?sN+OZF+SbR9GDYYK(&q=|6BdCe%{)>wH*_{lr*Gu8C zI>A?|V%4b78rF$e z8o0aSWbpP??gV2K0$@OD;l>{(>2_l&l7~EQSExbar*W&1Btqa}+w15^;|do& z{YEd8CVfR+4K%h&OL$H1_gLN<+i}kqSXEoOiH^SRk+RGDOM|{zY9eYX<`*mG&`j*UNJ8}$>1OB)S21%!rG-dKv* zWhq}whnco>FikodA}|PSh?IkMm#=qL{fi$`&L!`!Sjk#lZCh_pmta2h#iXPg2s#-h{sZ)hD6 zH9+*9#JWE&s}T^awjKrjomPRC>=P6I)dgjxB-!XkvjV}Ae(A{kwn_J|naLDX>+gA_ z%|Dj+GQTDhV5wwj3mkk6y|F7J%XTlB{%ajvB(wr53qW^~4I&uXqFW1bWHq-I{)JjJ z!d5W&;MKf_yY!#-L;b?%f>yjh<@!~Wy+Cv*EZ^`XXq!$wpu3#yDk^M@;pEj$k-ObB z^ZFnvzFQdnN+^cdW1mH7*EYjCp$fJ~igUh^*-X6?j@K!Ly2E6D>pu5Xl{se_gpqOQ zG6M1S{)_RJZ3auLl#1?mrY|ZSmJ>GIH$~IR?e7_4*;eIlx&77OxY*-={dGls7hB~d zoY%-a3Am~%jcXiEE%XI&^f@k-d$DstcDP#Chv)kW#0#QVwD=5!d!o z-~Jj-2}g$wxu~Wx<>y@LV)9&Qdvkn`D$`WG9tt3;xm6cu1C7_!A4|~EOQ#OqxV9oi z^T_rN9DChkJU`MI=NNBdD8=A0)FF~(TqRi(C%k>~lAq6PSH-ph{T6t6(LL@o^>5n;#0{(8rNdKiL*WTTv_~Zv7F&!58$n)4BZK(bjLoI zxIzr2eI50dXGlX=d^3|hKYbf8_}Ps3Tgp)@SJGkBAgpTL`9Eqq@2IA-eUINe-Vq%{ z2Av6^T0n|o=!BjTq(~6~X-W}7=*7@Mii#o-iu5K%DWMZU2pv?KK!BkI2~A2sK%@o) z1l~R{&V6^S_tyLCv9chr!^zIs`<(OJ=lA=5KhLE3P0^=^U*w(cVw)G$86GVU^C;!K zC#1;(>pwOfHgHoB6tRhhd#+~O1n(tct1Pv zvMhxS{SzbZntd%j39{y-YrvJ?{%hfPvDW89gMLND<$i-)U0LRWXU!WPzq~D0R<1%! z%u}9B$us*Mx&XO$-tl4*_cugAYqSDyt4h&*^YiykcTP++lUMU)_RV?PdY|09OIz++ z*pf&yFC~T_t?I7U{%z#xX!T=5e>;9sngPD?>Ud(s7r+TT{+v-i135nrfP?I-B!herv;1EVq`oV}E`jrDVb*d?xfF0)0k)vMg$OOo z)1ZAdqg`euA;*N4J=cG)1+`k#Uv4#}efI+^)LWxl|FK3L7?zd&J5t<_o|4LCC;BcKIxnjnRM&>B-u9h^+o!J8}r<3>uwK>trLT@i9#U*Md;4Ms-A za0!>R>%@Bwu$=ntJIY$Q)J_TWE<=1iCqm2%4@4D1B>*Ut-mQ;~s?q^STM{T(e_No#0EY@{FK1JM?qPn(E zY@D`m9fxE)w&%9SWWQ(gTa0W4$Vh5bWTB0wjaMH@-ae^65hA93nM za3SeIrY%_9v}BY(Q?g2%YTrU8)D+at6!#ERE2}ZxwLopWW*#K4D15%goX<23eA(XZ1yGqd_v#5$teWD!(fGCQQ`1FxO&+!$@%61-@4y0B zTgp6oXzf*x>78?JUM*!Gn!a8l#7RK%7`}B4?W0Q{Px&Zvu#mM zj(Y1A$B8C^qtX?DtimKkNps9`N((Pj;THaUuKLusc$NbyBXYlkX9 zkFOgA(u^owGjijrQP)la%OfxoK3*huJx_UF93H|Dmu05*J``s!8um+5)@yk5u)eXA zk&xzWR1L@m`?rc=KB_m#s785+Hg*sYY|`WxTKw1Nx|>xei5G`V(} zRL&8OPA(%em@Imy8~Dp{p*KZ*Inf^0ULuB^Xo&<^9DiKsyJkD9_tL`J=jY-zw_oVL zT$8eYsVcS^CyPI*Fd7pyyZz-sTxtYwxMx&G@!}#xgmmExnnng@?!03TycGbX2&Tw5 zU`STKklw!DgKDI@_b4ou;J=08bI12LE15Oyz|17S0GP;*Ka#JqiD^A2;{hp>&6@!& zLo20T5>%YcaiVhdbb9en`OvID=yR~A=+j$sK?Q!DO?@Fph?=nz8ZDRbB(DLhk``g& zVK}JXx=#L5k|`i=oGPdV_r;vaJlSk5l_K7ER>wSIp|gR$BP)MoBdF-5oPgh4ZLjWo zjwk7d^x1n$CSKyn3uKAMzp*b5A3i6ZYL`$@muVMIrw8o-@2{9uA+&QI!M%VAleOypB5SbtO$Jc&b#Ac0vABR#@Tyr zhGD*JH=;6S!b473z7)}0-C~XoYTW~ORbX>iTM^QgM3ox5swo+~m@BOS6Og#w>DB5^ z{X?cay2Q>613%fbGqLG#mnL`>M2z;jLf)p4Wq%g^m0|JS?!~*9HQBTS>r92@6u^+= zySyJ|ZSQ3RoT&rFTta0(4&pHJv=&nD*Q#29b%L~FyKr!!dje~x(m>B?!8TxTeA4Kg z)r3q`ZKR%4Qn#lmW!NCIkT|s(9poRN{TlTx7?s;(z8Y>+_-xhl8gkqRO^2EkBFs9$ zAQyO2p?sS=kX3qiyKq{@UoBmP^;_0h$rCjA&ax2DYr4g2Dm?HWQLAKRc1xYrO+lOp z8zmW)X!a7l@*pTE8%Kasd`oMD~s*i*4L!MZOt3zqER?LzXc3ER`YDg3RyFshnNsPy}==r(QD zviUj)vS?-jX$L7m>cKMK2QBsnfre5r=Y`%u6t5+aw?3c`_XLvm^Ur$;VR$WesY6GS zH0Z+OW~f)&;OA_RUakL2<4P+{@nDDUhz9_z!qRZnGnV1j-!g{Zea3Qn(=$@6=9u2( zj=i;O^@zF+VtTvqPHN`581rACLE-t#Fp4My8f>dp zMAvq#mvtAjV*>Wt0MB9P!9~%KI?{f(cY}Vq|62Rtp?|Ao-S{kOa)m}~+;nUtxm-M$ zJhyLoOGz9E5N3tSVy*}6CB_vjBI+}w?y^qYHW=axAnV}|rbTm#x5hwPa1bPE`A%_! zoX!N2%<9-1O38vPG0(>2*3KHSBcH6kzu&U)B<@++$+q(gTX2?nOw|B)-@VP>ZYKAg zC9J@aWcApa7fttGMLub-%9p2pC|v-KE`)MDdH%@0Na(y4qsDDA24$rqFA$4|u`{UA z@8q8EU76&^+KTc*zb$f9fut^{gc@K)FTtdiCHdHLOTeAn5m;
3w=}iZ=2ib~^8-FI{Op8Q;%dA+5hN>GZTy<%7Y0fFM7O1?S4Bpa#qCU@d zE3yGQcD=0Mr&op50My}3?VS5~k8e24`k7wPJ!QZBdMjX7+(l+1o!A;v&v?_|Z|>P2 zyj6@qt$a!VAVz5GAh(bratIPEN9Q}sA&5<`BdcQ$yY2Wp^X`$ILJ>1Sbz9j`>jNSH zq7;tR+{bcnX!yzM9gefU>s`p93E>-XV&m{TP7o1`yC=R=&k%2{UzJ=g!|N!6abg1+ zke4odZ?6Duv||*e$kG06)ICXJG&SnKl?_SttKXDm z4UmZaoEY?A@qKz5P~fWq;|ESCLIF5XBLI2p2D+p--LeJy;rJ7Bv2C30=xQ^F}pOe;|4+qDn2ir?cO^ZH~n}huTB?RiWEsnOiC4kNAH}}Dc2NHrf)gE z|80xumNB#|fmsalOBG;NLq3?YkseKyyT@ffB5_@2_i?9Gq3#3JA`=Hiq``M8_DWz~ zv2&&CxfzL82_O!Wbsx>M7%BNovM27DFM-bWU-@Qf&Y{?Bg~I% z-H1$c9Wz{7{nn^GRH%pBuyBbO)&e_C-G3(Dy-x)hZZy7sV;tVE7WM6I`+UvsEzIP< zG)-^MuXXU>^}l5Phy(B(A%f$e0{3ni&_&gSupQ}}>PX!%iy8b(q-ek#@bQ!~YkUs& zpele>M*x#b$SFaGbuvg{DJ}Rm$gApnOc>qM6}~YfvY&dWkZKp~qcJKcJ^eyFw|$e; zP@>`9>wTEE#WFy8Fm1D@!{0>NxD~`81zFB}>W%Cng zyS_3Mt>ZsmdzgtehS-$QI3KnEB^Z6#R8d;IHM!;S5wTGb5F2eR9$@L;-6wRH*0+@l z#uQiM9g?C^y=8@$dW+jo)zsa1<@{safsRntjQOjd_|>|=JBbx+sUFVw@R(83^yUJv z|6iZ7HMw#T_Ohv4j?}Q#W$m*EY&eD4ZK2m#vyj=#9r9m(!hXidv#zr+c8dU4(D_A2Qkj9XNfCwWA{^nVo8O+|Bk9()1X8=8`6O?th$uVu*jL$6FxfB6F{V; z{M{VS#!SQmBB6oV5Y)7s?qy8<$6*sT^go*v6zS#V{{*DU5Fbg$)~3ZAp!YHustB;+ zNihHhvJUMLYDXe4U10EWwKLV&M!u{R_=+yzphcqzv>=a(%szCs{253!J0?^HL8oU6 zpk;QKHCje&TaF(@Uo)$r=U)@fUw{y72PQH&)x^2S9Kz?a>W-~JG_*+HKXqAy z@^32Nxn}ajcAkY1mOC2U;N=lu?lxF7G~`S`Vb}`zBGq6>t<(|3bZ)iuezN=Yq8rRryBJaUBF4IRDrD|r zG>!rHFUC{WxNyW*xT;BqzWImIe}zw5@fS^*Z!a+q9v2Bw=~aZurXy2W-zRvPbXiM% zO_87sCyPtw4drPt~s* znpZGqJmx3rD1ZPa9b#Y}os$RkgVUzYBj!z2^9rA`vdpb<(;Y$5691UisL!viWh9PZ zub<=!F{5I+_M@7;K_6PL$#Jb(p0{`6XF~~_O2tmcM$+;_D~h{(N=Xj&2&Q(MVh|Ob zNvKWgsoWSC{&%OcH-z*U-kfjQ%sk?U``^La;MNVm6+wDUZQU zE}qe!w9;?0qm>+3@{g&G>OZSgI)UTtadoOO*ADV|CTw2Q(lHs1G#vjNx+W0JVeCBc*evxs2`iisnck6oQ=sj+ z{01r{uPa_9u!04Qs}zdHEF7Pp@W;rk6^nH%4@ai~%a)PvGUlS0uSI1MBTe?zuJ4`2 z(hi(s>h0p6N0-w|xs^%Wm}#jFzfgB&*)bt`2m; zyZ10&8vRAUH7iHxx29WOnc~a>1m%s|Q?Be=TB|w3(yT(iajxe9S1AJV7i9-`wy3gd zyVP@$28K~Em-Nc6mW#PeLDAOY!^Yiu0ox7oV;sY&x*A)mo3EnhZ<$8_F119LzT2PW zq;~b}eBcg$po6k8pu#Trouc_YDZ_GL6haX0A!Vg-TLl-6Bqu~F1-+(ha&xR9-SPG* z*8=y!fu{*#u@FnFg-y8m%ki0?W>nw)*34mQuVz-2ir64<1mAb@p#I7_thJzjs&j|8 zH^(PV4YPd$AX)OvBUPN&y!19N5h5*}orMw-n8b$fe3UR0n=}jTNI3i)dmHHq`cXpy zdtWjSr`0cg3UNhj*&eRVy8^WqvlZOPB4!Qz-X~)XOmhiI^4sOA~MC{K$f-q~|-i*f0-lE8`tj!zXEJX|KNv>Y@~)YVOY~ z32VQ0i8!;PY=-f3xz~{qa7ol`r0XYbM%BdV5}^R!OeFo%nf1yF(X#=Da^)hT+%Ft0 z`-88oXVvME0F5fh)T2x!%Plo@TG9cv%XfH+cw8Qrc+_jpo0c)YMQz$!lBz`0V}IHd zwmq-N){+DJTYKq)L?m5$5ua5Z>ONA=UGRd>YKJ6iq;0mctj?z=YqpYGSK`r|getnY zbRC0r{%OoZR~xCyw?jxnZt5dVwMM+&{6T@m(|Ck5Eyh1Dbe0&c>fPlKw>LNb?wcD= zw1IQXCZVT1I1NZLm(y^BZC#mFS%<#dy)S7;2kex&&~lF!VC59nSE4V9D}En+a=;&nDc-^N_!e8S+8-Sj;v;45@bN6X5y7j3isPMIHLKw zM5o(1%bvIRbkC>F3OuN?#>K^hcl&OKnCkY5Q(dy1I65F>v{Xl!@6=Rw>{k6pU%&Y3 z61Y4SLUdcP?0+4C<=l?Jp^leAY#n|~R&|_qe@j+Nk^eVaRv7XE1{)ktvw!`;B|TD`ZhI&G@aLLBrA=+u>)LJoYV8cITMcg9^oMpF%LLKu z!=`qkh&eU%AZF5R<7p>!*)6#Em6`I0up9iqWYN83i{NFwj5~|eF}+>&&7(AWO+N;4bFKBh%X!-5`#~lpXLJ7|(S^GR_b1Rm1^fBH{G}=iBufcgH_thS`(czDPC< zcmEMLApB}n%v>T88Sv0;RZZLd9sRm5)<1^YV=NG*v(~n4@uT~^U->CN6(9ICT8flY zns=`|)L3{lcDX6I+&!nE%S?WtfEZEVCzQwQCl+C^i55{d5CKU?8^0CA_Kez^Z4aF( zsLb*@1F#)3GkdACvuAN?O^KW88SK0J7qG8Sx|~Lcjj3l-5kh7!Ob0NWNUOS-hboBi zR|6-tiL6yO&R;>D#gzn3Wn0wT7_i9Z@OtwnBE|7M3WKC&>Xe#O3-HpN9O-2h0*_i7 z>sDEojA{^!$>#jM-9qL84(vUImXnfQKkxN^OQee($8xgqtO@tO#~hx`&u+A~qn-#Rai1)Zn_NZIf^qNmW#@`8k<*L#Rakfhvp5P92;8ZZQ*PIl0iu zDohNguCT@;<05H7v|XzzaOTo+L;zEcq-C59kc+s*wGM&YsPeuF zq{7Gr;N2!!R(Ibd8#+$yO)@_$mP%?K!PP+J8xM!(5f}Kc(l`(k-8m#$Cx28QpVrsW zyaI>-lI4r5I^2cFZEizB?x2yU*$BT>Y@;{T(*BrG=N+b{fQ}2ngKs0k55v;@V#nQP z)wMre>-e1?VoVBNgD`pVo}@_T6ulo1(()Inils5~yHwobQcodE0bdZS0c3?2UA0BW zUr${DkT&sUdjFOR_#+fSdK)HHn1L(}>PsEEKLpeej#{a8X@wlV6#+!RAmIZgI9Qb- zZ##$32kB2k9!&hzo`D7J4enEtUF$soY85?mpwM@>S-(#egzCvwk0z6d+x02MM0qXm zzoYjpVPQ5jST-h?f{N5uICzQ%YccQ#IJ)&c@WFuEBAx=k zOUcI2a`pe-Ocu1=2sTFGpaA`!I1G$_GZac66}JBS2L!zT-+S{vZ07LAHbRgS19DQjJaF diff --git a/arrow-data-source/docs/image/core_arch.jpg b/arrow-data-source/docs/image/core_arch.jpg deleted file mode 100644 index 4f732a4fff5a08bb7450d222001d5adbcc8d5bbc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 71904 zcmeFZc|4Tu-#0#zh>$%?rm`mq*=3S!Ns^+7sqE_{WH4q#vJ*lf%93Q6WXn2sA(S=5 z7$mzH>&a}tv##rVf3NGhpXYwv&-45JUa#kmXE?p)Jm);lyH1m7rha}#j+2&PoI(D1Y{o<10xp$y%holX=MUA`o+|rFNS@L`wCtSRyf^u83*LPwE3c@m`dD4l+|t_C-tnoktAAi{ zXn5q?=on#oX7`ty*-9$E+fX%AjPoleRN2Y9AHO>+!4l54>ah}A)%o##9Ac2)Z!)4 zRF4RYa!2zb-KPB_=K4`)B#n&=w6XZ6`>!5%L_c6v>JoqPgwQ_`ESuP^m|Rp_D{q!Z za~hu6-VdC#rWvI^#u1rZ@We(a6K4K7U_ggJEE?>o@ibNB&UPR*Ra*{qhlE`*Awb9I z5JE>tor3SQuqKur@HXxDWpONj;uaktxZ;GoGM{t)5pg`@r!%9U41p6+k0)Rw;DpSI zw-iZWl+gDC%5z%oY`1BIP+YJWk=l@_c6BGf!eb$ty3$KU#j;j0qu!YCr z{RR4|oTWHIur@0l@>Sut7{q9e7^bV<#O`Jou&-hYT3JQm@4|130jbln>~`BI>MNAa z6o#ozb{cxrCG)8Ht8b^Ggyl7+I+*1oNgk-FfsJorCVd9Py2hhcRlY z&aitoY2aJN>5#)?(E|kPJ;SW412gw7-@}|rwshXEzPxHOe)GV7CV>FgeB+?8dKg>y zSk=}5rT0F%5yy1tCyg)lu_^Uq@kv@EoarVfP^ryDhcrrH+u@_k+Z(`F^38FeVW}NT zG?!ub}-)cNkbu$g`IT>U|?yH>|e9Focp`~(?GC1NN% z-P4lQ#DN)<_OY^|y5qzxEz3OM!<^<#cbY^wOM)riGcVjKR&?e*S<_>@+uooetwwf> z4w0qV?CYh}<9hKgHsvX2n492xlE2B~%b^`%oWfFhpufdB5{~kvLqa|;;2?F3#LA<( zQ*=n1oc9cKTfL%*lMw)ab<3at8PeMed-1d8+vM$&1tB^Tp`N$VD_wJ0fIPk_2)Zpk ztgQrunvgo4%o5k4@QoVrq4MP+4dS7#&acOf4||UFGig>KSoggb*LaqLOUa7Goja%F zhJXQ>Zn*9|47kG?g4?`|5~D+cWexsDD>jYd0Lq~U@fcGY9*0&$=}=#y)llkX#4Gvv zF8EPV@%L4wHN#BHA<8M=>Yh8qWM$ol*h6m4a_Ru1;6VGYXb##aniCM?3`X6ZM(`rX z8wDE94Om}3qxd=Hi%r+5WWkn-)+b-Gr#vQ3ddED)z{BVeIla#~>~)+A zCf~qb^$fvUjA`6$WlXQQ+IOy^ZdIjdEJz-cN>SKUqdr2jLU-TTa)VC6l#gkFp!n#J zY&!y!!eB*0j3Ty!s5vlu1kaM_S%5^oeUoAZc)ywVvrK!FMgXAOctRqG_*pfV;5Bcyed7~YZAJg+SbyeytS!O5{`d1Xb& zp(Q0w#d($?LVW+drVFKJuFE*=QvL&kgZEP3-%JnW_sIc3iVnHh_dEV=7_9JsBG@2~ zG-i!?Pj@u4t4`tDIU_RFfEykTZ*N;6t&Oec*WBM!-I}XCExf~Z{=>y0ho*-K7eW*b zLg|pE^Oz|DhT9omrn=lmf^B1n@-fN?cFJ9nLvaOlqBowe(Z{GnZuKmaaD>gJTleg^ zuIb#o9ir1C-DxCa;Dl%`(93tC`ob;>Y%>BUTjHPm;lHnFpvt}I5b+L>%Ka5nFcC03 zR%)0wMQ1SpmL`A&I1iy63Ds zJDwttL@0unbu{Cn3kjGb2E^`;NhxiPk&5(>*_DX!faghF>lHTzce0re9g3_LU=$ZP z-$z;5sK=d?uh`@t+l6mYDmJwrW4`fBz4HZ}err@b> zIOiV3b}lsyr8;AJ7!YWKb0~XUlN1n=kMPi|-*n)>pUqC(XB@jg>*jc-zoNPTYMW`PEpXXAb>1rSie-Zul(<1Z3{^={$DEo2nYXH7XJ>7|KG?0riz&= z#g)ZBLNif>i7(eeDYDss=lYJ7vr}68cm4H_Sxpb|ho6{<*|i8jUX}7Rem@=Z&7Th0 z%)BJcIN$e;jdr<;kV*~q051su%7FK(FVJU^4l9D3Wqy$=$!tsMm({zYq{I~J#cVJj~ z1_s-+I0txWO=T#t%m84{r$g+4H%sl?Y30T5O!CLIo@nyecL|BSw4 z56Ww*7&>hls*Kl8*Np39H-21!(WTR#CbUV!8mQggXVs%)?y3S zBe4gKffCK=5a0xSMS^^bHiQ$+)jkNoNwNk&0A@ZLor>IXcQgh4j5NJ%x+KCu-llX1~1`2^*g$Sz%8!Q zHrAmuR%TE%wE&*%xp=^U=+-wQ&<=j_#<{@Hl)(#U?W1(;~vVmochh(MR~Ysa`7(o1-w#z`77B9(_{g9 z-U}#ygDAxaKoIc+Sj?M`1M;ZjcO9ma-kmO#C~i#itxhn@3A=qFBEoJ%f!p1zZUb6| zUp6P-LR|v|RYQSLU(=M3%#GTdUno3NLAuSUHJEV9g{qAOYme1yf~Qrv=8U|Q2a>J5 zPjm>C>`Y|PO3W?>4Ua8hdxvq~FX#9=QDyTH^Mi zkq4@aTqn;G<6GHGocdmFsjr^O?C+iK+ou;H@m{gvYkBw$#eKzAGy!Tb&J8~c>y?ahk#pdDY@llz7)X!aV&N%_GfA2y-JlYVLdgfDXNimWs$}V$y_5#zP!2< zWj1nvJ5A4Y(nPz8;z#ZS6ZWYDe5q8c>8SDcQz}-=XeO4s&v*bWhcc}uNj>@bWKTF1 z_ul`sptzSXe|)6DC*aOak`O(j~H+F3B`ucq0f*YzB@u`s9YqN&Qq(fd1ogVpM$9A@V z5-npJgeXCPF|n2aZNc#)&94z@c?I({nILP+%*8p*ld|cCb`#vKxz)~dzV`(0<7F&% zWnfb&m`cA6#HcC(vy34zw%`t;G>I5ONk#tmMJSg~{X^sQVn36z=XLk3MxSkP&$vII zIC-KpI`P#@dJ=TBS{=QrrIUkGff7+x{bX1u~N%*jZ( zL41;)i?jt?&uFEVH97H~jy;S~JR```9mWj#IWTt=N@E!z>pA0yy0FnTLT@Tz?NAFT zXUekeIEkz7+}kCEp!Dl;!aWxZSNi89ok8p=tik0l69?OYM(mjDNjuRS_InaLDkHti z5+!uVa1<;N2If9fuW&>oXf(duaz%ZrZ|s`sBg{ecX}8G}M7_+gnXE9h6p7KgB>JsB z)Ig6+6{9b%p zE$-mbL4P|!z4Yb8bT{Ri&mR_*a<3jwwnN$>(+}MUm0Hx}Gm)&e9l^~62HaMz-vg_V z(p}+cc|ZcO!bg&#v2>EWw{?ckQki>++IjmiY+5%|@(@RH(Pxo}u)#C8JNWIZstv#9 zoFZi_AU|}teZuHs?i%BxgsPV72{7QcC`Ui);LNxzq7=R$NX5fqK>47WRQRjKh^dSF#94N4kVVa{u61k?b z$eb`*Ql}o2Xp7iAE_#p-dA~u2ENDHu%KSC!y#VC}qD7tz0nv|nDZ@YCOyp#QqW z3o*ZZ$z|{=Lw}U?t9}2SBgawVPEsijoZY44v8OGkD;YhYuQf|`OXeZNA9_!WzzaU^Fb+(@ zU^6>Po1Re=P#%wM?!%_trb|qzhtlHd5GYt+Jhs8i%T8k{GRjaI%#wBrxu zX_$^xr}-7W-3~g0F>kR zm=4n+5%4HFB#T6Tv8S$w_QEtPL8(l2|K|1A1Da=k@w4|!EEv)rY}#WTvj-`pLmpj( zO$$udjKqKM`#L7O5l!O?1K0TDrhFKJ(OWu%v}aCu4Bw4Lg155%o*M)Z{)@Rmft`L- zNr$)s|1sb<`5;8pI?{F(AUD?6P-hj>eZe>&!W3EmI$$?~`2v@o*)|MRKsnimaz z*~kbk@|TVbC?~q(cUh5z12p~LoP&(;cqL)JFqu+VkN;KY527ewYWeuCst0X|WNEr5 z_-ptj)ah4r$S3oESxh{T?w&;p22E>o4BQ<=1G&>>;}A+KAQd6|D1s==RQmBF## zwBO->PTQr9?S%n)skD&4I0>ft6mIjcbV$xff`kZV0lc7*|9f7`pHNV?bcoKstqexs zhwp*1{+E^gOV;dv%DPJ6Pyc!l=eF^S;lDBnB}V^~3D87v^9S<(@DA(fe|bmGpAPoh zoBnj=`2WO4>vwznCZjnWvM&PDD%VDb^a0e*RFS#=S5t$I2|C9=1oRI+NUDOWRLCH> z++;&ZY+jfR+OK$|o|E=FruoQCCN zOw&2f5_vR(>2`GCczXqjqYxoHal6y&SnTQ9>(^F|5A!l~o|Byzc?2Jzz|sxQb~`&> zjgBhh6Hl)-DjY4_KsoXeN_kp3C)z47x&+hzR5F5@<_4ydg=k6*2N!Ik+3fnH=X*;3 zsQ|3Q{CNq~{qG%@`jB8&>FodhF^h&_I@w57)L?lWSgI}+CID0r6O(VpZ$ldBkbZyJ z_%n2fZvjmZtzV9bsJEO$L8q|0+?!c@q$+U3!x%1yX}XH@gbAR#8>9jADLm76q;5-E zYTLO>bM2=5K#@W}O8cq5`so9_wf^4)J~5bztilU^L6;{Slg6nB@N|e?F*1B*#dpT& zEW3bde*+_cnE4s9rZt0g5vX~_zf-v$o6Y=4N!=U4r{<7=Ef2V#}Kr=KrVRX_H*q*-;WSV&t3eGr?15+I8K zKc`Zc2gq>b_*LRb$`K%N#+1p%AP1#Q8>RKTX+8_~PmQsZ5j=hba>g)K_(Jv&G#r1Z zK^^@aT8dl#O3b41nPfddvAxocpJvMvJ?be_eEQU}m~uzi`}<43E2#s(0valX&5#cEtPHz_!q zCuq`HAYS33)^0RETdw^XSOfDMYrvAMhB6*RsZLv)Mi?ImeUK6`Th&loT$h~p^zc&16+LI#R~*qC7Dk7B7-_BLWCqd@+_#qk+9PVV6Vvold>M+wI)8K^ z5T#XG>#8SfFL>-v-PfQ+t;RX5jTz6ZwQ|N~3f2%Uqw8Gqk#BzX5U*4S#Lo_BoqO%# z#UouAq%3~@$BAw!cAqvz{0y#c8$?Z0~U^wCFYYk%`G;g#`#ImtR0sl=K3rQ^zn^y{uya_ww6GtKe^Tat5sh`M-R z=vA$$sC=;KKkRWz&*~EY?XN~3&z;+RNiNiO%dhfyCdvkSVQk+NbX<2kq5%k-4$?IN zOr~J0^J&8`-q?)liR_`Oyv|P=>Y0h%)-<&2c2c{ie_rq?Vnq)WCX}R-`igRt2nIOQ z1DEjJ4Vtw^uWupS$Y6oyU)?MpZ13um($6gxazg^O`3*e?Vk-6HbjrOXFGV$exHJJa;&yt@6XB47tw|>3gRK7j7J3f zWI%gngAxs91X@=FS}ar>o7)<&h!V!fTvp#zC0;}@em2}ae&tB8 z_f`*LMrvh;XgY1$g5^W;HUP43favBOji40lJ0N9}nz?tG!T5bB6CxT3CZ ziUyBTiJ7MH$f)mz^Ix(2NwGqMGR@DLbc>mQ7G5rw{c5?sqF6cjsFb z36$MFb(6*3@uuZIeXX~%_CJRz;DB^~%Y<+v9ddx8BY}&=mC-b8nw$vlD;0JVI;F4S zTtq#^;Lq&~Pys?jcByR>;3ag$nCv{?kGzZx!#SHUz4?MN-=0tDot(Y({bFUm!RJRW zR=;sy2wOimdq#QImLFgs>mflagnoz*^TmavY76$G8Ifpc!e!>G5Ut^lB%srqU{zLZU8ip*R8tb?mXxg>WN@k>5uO06OfWvTUjPIPjdewegH zKnXW~$Y16ids*DGLhy77yHUOC|1f5x?8n?h`NVQcVvl#SjFCiFM%wMTm6I8fw&aiG zk;(~;%-3KE%w|foCZKDlS5EKALnmC(OvB>+%C$kTmSW#+E7Y zn5nc?Z52$DAx(tBPwaT?>rhVWh62xEP!A83c)(vI=$d+qVWvRI>PY$N)|$y2-zk`=TLKc@cQvFNdZ*e;!OkbH>#Ml!#YdmM z>nk!crjbm?OlhoaB*y8!XqZ#I)Re>qQXd*)HCK^cfsfHr^t&X--7{TnXsyU{^YRaC zhUaIxV(!(cjDU>{^_&!*90I=ir7oejz;~LHiQ*o8Yj8$@e`6Hy;(q$pDj})o^V4{R z>uanXe!9Kd&%ucP7TxBTRoStkLsqwT6&0u>IknUa-y9lm!?J&^vNm?fDPQl_&@faz zYe2o#*X5Sa)KbTj@ zhE>d2GcV687B0rw7PVhj{g-GHSwam*JIR+&CrPWt1Uci5EH*7psVE-|AJVn&Q=E$Q z^C)L=ru|V47Z}W94(sk?50qLakORH}f!iL`=Y6f#J>EA-`5o3bPkG3D6cL=~8cv++ zyRUmJ)-)|6y9#}BvI4|fdx1cr)3y#!I)wLFT^VnwW987{~*AzE~?8@CDU48L~2Ta_ZO zY|C%pl*Fp9-{{@Q-gX}iRyH9O08bm7r;d?;)M6rK3&%%pWCOPAYt4lShobYXj*BWT zEzh}FlG7chvQ#F6DX&SKO>_u%13zH5JqFa0bejSYE%mTz)5n{i%0I11*tmWb4@xO; zojYdStMw@%{f=J3PSB9J2l}(nAzNKsQ$6SlKYr&b8d;d$Jna+gF`e15Z z_K)uU%UHKI&l4SFJ*HL3ABEBVT9i`Hz1*1~_mVzmKoHCk_=vb>#As><<}k%$W=#Z) z(z;*K$GQoX4Ryq}H|bt^n#pXQ33~UtCF)S6FP^LKr!D(nuTV2FQ*ZFBbcjn|oGO%) zq6&+W13Nl0e1p`MghXn58|v*jRaHEsdc^HfjWw?w_6I+#)P|D$mf1+*W}sm2qHYnj zY#D|mNbm^l(<->TZ~Wi;rv~`TlDD)SC-O57GLJPB0QqDXasUqA-i+b{BHak&?)d#k zc%y2Ke%`x6sU(L7OK5IxRnN{1yZKyH%VhP(^e{j_r~SruIJf|HZ-l6;Hf^-#wO!-&S&i?ODqOep#>uleYaUAYgYehz_w+0aL}i z4B&?PVxUk3b5l- zS-U(w@qZRzR?nwDP(TJwJOn{CTNOHFPsYsx*N!r;xD`;g*AHdgJ%Zsvi_GKr>zIm= z@Ky`i0%y@<<%u0;_J;ftC5w!M_^Rj9h*D&@-h@`mj%w(TOc z{>iOkbma=V<)OV<8D;LJp0{_I@l8bPihi#_g+_>%ffC=-FCTM6LKpV>Q^to=&)S5a zw7cN3(8h2EmQ}BL)-k$teM`7(b793Gl~Q_TLVr;`S0u;kT$cg1UKn=eoXcp=S#Lvi z{pIP97nA@RZU(%?dG6_hSo;aeyv#>BUH$Y%Jo{XUAV0DhK|LxBUKOkqMbIv;EUwv<;zo-ftxnPbjh+o&l zeEHj3;kOg#nRp$4U)@9dFs4DIv!ft;nzVx;%q-4LrnP%G(ILhT*np$5O_-^dxOE+Q z+_K{^X0y_2=XF1BB%T8{o6=2(EPaM;KEs^)iCWQ+Sp6*>~Dl7Uyz_s9*&~(z$oofVC+9OUwX_ zP=y9NYrZ^yQT!-wQ@BVifNTjiCWcaF1$Y_uj?y9QwZxya&;$@s#?2T|RMuLu_Baga zkT^Od7LA;axyn56_xGz%;y8{2Gp`P%Y0{K;XQC9&Lto?na_uap#gn2Ak~@n5)_{?p zanfxNbok#ce*OEj@JzLw|HFyVp@MRZ-;MjXWYQ=ydqNPZ1<^0so#IRA_OLc}a?@;V zm-HQLJ#Z4a7iyKaNPC}Cl{L%yVsR-D0+)HB?2ed%JAtWh`^4-;%HG;Lq*tsT)>jD_$a@k z&2Uz8gN`1rr8@ZMDuW3o!1uV#;1|Kn)1-4*!4%ndZdRd&l~y)&sg_Ivfv22R6&P`R zKViggG!PDYg^7U?4di^F<^4amACg)ygB&0xp{I=6Ir72z*LP7GA0xap2!2&HwhAkw>b z1Tz(91^e3>>9biB4Me0P1&C`j^gRsG9=3aHSBi33ikl9Rkf$)vA?pUEm=-&vCG8On z1j3p`z^ez?F&TU@NZE7;KZoP!g8m#BY=TmW@0`MKugF ze{_bE!qsi79;IMFF~o&qHa5WUA9|b)>1v@#@BD>5#u^{?Aj=ce5ziv>rthgu$O@Q3N?@ z7Esce3}ebO9Fdt0=@jh%LqseAzQ+u-fEmSu{{Oz-<$vit`VUoe{?>cmKRH+UFE|H1 z^&gxY{hgGNxak-$Tyx74@DxfbSZ{s8tiJ~Z`FI*W3(gdbqZw|?|4rAaeG78{945ZjJ7%_sQ9_e2Q+@LUOiF(xhtoXpB}P?>Y1^ z&nM-Gy`14{EefAL(434pEw08Z)fH9N8re`1s4h*_6{rD;7RMun-h&}Yl_Nn(4OwTh zqhvRd7UFYo|7i`a6Zju3wp1i>hGyiBqt$_=8yT$kg0WQWB)Ej7&i!|Gj4j7RjW(T@ zK_8Q}CU(#oe`Rg#0RR~v=Fj{;x(Y$xLK%YD>c3V_{>lb%%qMCg7?PW4$9d z$PqA09iKC(9k|~EG1B981h)ZtjM$Ug+KL7Y4)yCI;#Kr6TsxJ){#osI=7E4O`+cr* z@(J5sh^@$rwf^${*BEU!;plYr60xdG%VnkLd7S^MVgSEt{~eV3;w?VQu$VrZ62+0tUVu(c0cBjfhvEZpfNycrm|=aJBW^U`pZ57^16kq4sc?mE;rphOTc5D!fNF4n9>6n=3DcGya7Xz#)(!eZ*E);QS_dsZClGwi zk`Rv)&(Fo!TE`TGrRQI|LV8$kT|c6G#PP_@8<%`f6b-Eor|o4*O(n!10QW<;W)NU6 zbxMVjtvLZRRz=v(A_OT;A`b6*TaW3AMEQ(2-w2Twi>N(pK(NFSO0<7vG~IN_vu-?1 z2|hb1)?A|KYd?G=VuB9oF4&<%4pP-Q&V1UnB-_9!9I&|neHXTSbu%=f8A2cR5N`K! zJ!mRmnqo~+hixc=abFP}6Eh4Wv8r{`!fwV>cwn@P*sT^Za9Qla03w*r%M~zTmm1Yb zp9#suMa{sKyY_D4PgevawLJBtfcH8b*BCl34fVmOuDjmbi9jqND;6zHtkaOpQwB=o zR(>SP`mbaTMZ~l(_Mf0M791<+34ktB$>&X`**>AvE^ z=BK_BIJ0Ta9L(W6AEE5+*PZ)T-fiab=NqRyxP&T}`ZR0waNUIu;t>@>_t#^SgZLRBN43w6DJi{0Q#Nc)IE|NE-6%=|#U*x1?(^vauOVfYY^ zFAo^@&zyKm(Fa@#P;!GF9$PYJ3x@Z;+l=3K*?X*7xx>Y+|07XFcIc*eTzGMSQ>(Gj z7Veudp{G5WH_1gqw6LI~B;t{*2Xd`O>Um%@+TvIkv!Gzx4Qd>Piw_^O)pKd-V!XxR zX4WCwrGNujfv)!9=G8in?}SHQpnQgWl2DuVn!>nTk1CAFNEH{!&hfB$nN+{oK(@mB zuLNC4_j%i6HDjmS)MQm2*w570Tj) zO=NUX_Tcj&3q3tyQcQtuz0O*L`~qeiTHd#!bfii^fD5qfjb-)p9X}e)Z#HuL!2|U+ z@4EF|*9x_~uDs7!U!R4-{5M?z3)KUr5|8Ke1>K~CVUq7`mxeX`q2e!+jUAqO2x_-P zA;u14xJjLY3vg4^31U|hF>6Ys%0gCXF;gp3&*eCD;VSndQ?WVS&={h@<>BJ+4(pzK zBQy3R8qmjc@r@7Ive?g$$>CCSF3SsjH++7%7BWsl{TP`nsl+JDob(wm^MD25-4c9) zFl;ZrcHSGbFwZzuWawM$d$jPbHVh~vDCT zJ(`lvj@{OA%aZMiOt=x{?*vs4r=2x`B_+Q+mv(KZ($iDOrQ{v;tb{$Wg8F8sGTvIx z*v9AU_~7p5mAvg$o5>eodiZnRMTl^H=6+vZP%$B%BgYmQO6ZGJu9b5%k#Ab_^?!X6 zP+<##8GE^w>7K;zgMQwhRev}_?T=5ayTq=;vLHhuX zHEEpXyz3X^if$M_FgoJCGJEe${9%oO>jmaR;_X`VY?rp^tq5ABK*-(B5KmU3{M zjM_1gbsk%+c~g_q*>Hp+HG-C3QsuWgT4Wv+cc7;B^{*)FRjn$;0BbLonO`WR|E!KIFR6PFUON{W@e+fFBr~+)rQVDoQXpmG@jvP zS>ji)Mm`y`HQX;w{(+F9537htg#PF3qw<-81bgZ$9Foo!V9VRiGA65MtSS#}KwC!a@M zA#z4+9u+nv>Zjg)*SlRh|D}og_-SD$(_+_K5zRA}-)}yVYqO!IZ5EuRS`*?!v~-GY zPUudr2qJzEd$L01epXd~NK?JDquQ2C>B*7!qQV+%ByK5TE8(+H-J4l??L#tkIHN{y z$)R{TC2i4nfIth#NXSg}@*EyIThr;hmJb)3vG{mh>Qv5Aox5#5_`@_oIO^i zxo2OIIfl` zL06xbQ0WP!*qo;#l*JY){YocCPd=6uVks>y7Ff0m;8oC zHRY7G4T%P!YqS$c+~qIjZ)ei44XyHzjf(0B)mE{kXhFP2@*W1PEmusx6xJQauB>Xw zmlzO+T$BxcJRXi;t$wd3uGz`!Zgay>GZK25!z%LSPNuB#pH zq=vh3Z7tky30tUw@#NadmBO);6SsPT8wvS(gT>i%72ltEfIR?}lz2%Xme@`v#AZF2 z!+k7$p=FkBy)>NVln@zrFq+BWdttcnl?y-oMbhRhe4nC6E^1=g@W`y`_{Qfx zM(;@?G7Nu|T1UnZjb#Y1L-T6YWzTmk>OY zRjtNBC*^FHTAn(8n6OIbK5KjXRQIX715ebNzAH3IrDYwyB$7}lTot(MCSGH`Oou2_ zUQ}F8Q8dpu`s|up-$Zm^`QD}pR66>xV{=WZR*cB(^%vOURoXEV5^QFzZH)o%f`*1S zTL?X@+)g>;^7Mj-Q^l#y`yWmo)Ed5jJ8{BvUxQx6>J9BttmUcCc~LbP>Q-ifcq2It zIiJCejYIqI4|}Dme25z!R>4_*58^emEtm@fOFALxHQ)<#3A!D&ejG@pwFhFUk`>rH zM`gjf$PT)Dm;#olcIQTDNxyRLWEwm51Z?()6MT#2v?ouVlauDx z04W6Jsqq?Fup)~pmr_6Pg9Ve{kcVLUw;yaXDNVlx_L_{kO@?0rF-$Em9Z0kWt0*j3 zarc{1%Caz@x^Tdc-K>LPne307eTs4?;djF1$a8B@{N0Mq?ISu;(NnDM1^76=$qOPuQ(lr-2ANeF$${3IqRA(N#I zwU9emlKO7^cBdO6@MvA3n04>LsQWCMNA__H#E1xOdhpCtg`+qNCR`f?i9+*xo5TG& zOoL;yRS&lh=$%{>krSGo%0N8amtG}TuLq4tG=%`nO_+mq!qe;lru?>eWVURi=zvKY zOF!?EhcTaoF2Y*M_tV4@Z|S*kjt)1sOgGfN>c|H=+MbS$=>)w2_2_?CAq_| zEM0}UgiZ(I_@yGxvTS+Cx9NYYc$v3xQy5m}#hUvFdQu=H?~1AVQ5hLmDtul$Ys5P{ zecS5mCJ6HJYjC{w!n)LG=Mff^-zS#K1>NZs=8wDZ&JLx>v;OF*S3;)}urjW^fnzxxe%x!(mAd{EH{Uv$K{k*ouxbN-t-TAI%kRY9n>N zhPY>~TTuGA>6q+6o3Dw7Tszd|>M|(Lf5s6F+63#^r{uYvhlkhMVttDyh38l6@=M+P zAHVB*eP!AZg6giq=N+!8!3^VOxhItIo3fN^{dF(`9m3oB26#AgwV`TYb< zzJy7&=1`Hq50HC3V54$d;EyVb90IrwJ7n52ipVLWb--L?#MwONcGp6sUW^|_wkr+S z$xrvSMrg~ryZTK;bJGk0+s3VqXS0)QSvTdsvmV8xBw16&)h6n()IpP@sD~_ALVY zb4E8~-*iJAUHqoaMb*aV5DOi7cBpgI=iv86A{vgACCYJjq@hyN+*4(WChD_YmNeS? z;>AQ=!lG6s^q9>=fP?j128l;;ddx$1*pO*y+pY6M)+*!1Jhb z(aNT9?RdnTTd9&^6~~d|!UE4A?4z1Bgn;WXel++G3r&gxV1x?^rhxd^`zOhC$QxtUjSK$R8`OF&{=38OloxevW6g%MF)NA8wiXvX-*x%#54diId?JXLu5k8c^PjVD9-iyE zsnwVx#Wctm6PH=>puNM4gor%$#pj5{(#;y-b^N0Hz1~9`qFtOiL-C9y&OJ)Yh3v^g zpGzxVz5g2Ro$k_-Y`YVv{jMpr+iOVSe)E@ndH1m&7A|;Wmzp5cd(;e?<^-~N8^MMQ z^_e_2@QSA+kelZy6*s{tsVMf$=gxiMcYB<-7cp5C4{$8nU}F!9mh!f-id*4$C6BAX zI(7~Fqhs>i!?!d(5$T!CiXo;ZED_#@8V^?zm2R5`u7&12ewHO}O&)_gDh9k=a@djI zgonPrl_?;TW1#jn|CXE;g(t{rv>~69d^sNIFcrakV_o|L^M>wK>{))vD(|DDW8+}U zOd!6Toz@T-Pqm`TQV;Ec5Lu@JsEuzIF`M5_=is^Fl-M(vEpXp+;j%BU863~N5qwqO zIoENFvjXA+<)EDcatNniE90N|u9)h6tiGBgeCyJwoXQ_RteIaQOBfd{*aCIlaE4?V z>rOFx3#41sj+9e^Qnpr~8@sDdE3&YttFy&yWX%3_YtAaew-(yynhq6fivZErpQsu~ zr2M>)GS|PLrsg1h<7>F*6j7%MYe>z;IRs9a6V@oFupEG<74hYJDCsKTy4YpN5i8B; zvrP`(kV_YTnX`iqwwDUEKGU5O6_qOHUFTmvo?nq=V=*EBdc~}9=0|u^yrS7iN`INn zh>7ajH&R~&n4W0eA_cAVMQQ21a|`LyzVN*#Ir^NTt$19Hz>MEElydDG2nN+IW2(n>G5-%x@B%Fmv$qfB<|$6#*`_E^4-w6 z*BZ>^Yb#DoQ(iknhd6v5OASMm*ISzjU0X~Ms`Il@Nj1nd|MX%#Qe-y9PK)SPl0PV; ze8E*_bo7?{&W8;Os@D}`Qy5VCAkb!Fb8K%F#|UeMm8-^~ZoRuXxtMu!SiWSlwr_<^ z-RYU8RbQCRLPyv6tB`gz5nm||=+Py9l3TQB&i#VvE6P_DXfMfqN}xwRqI~D-t3V)= zrkEq^pgBMq*|MT)K#n5yRx;UdJF>E(ZD76hJVe`NG`x%SzR>AFrE=e>RJ*pwm$ed4 z{V-}y-nW-m$QR>pJTbBD`o`XKDy4Dts=y(GQw+2ALJ`x7-}IzEif>xUp37RyD%2y3 zpk~Lfc_3~1w&8f@A0UH@wz`+>2)dfiKUhG>*)PLeKDC47*W=_oDL14A z%W+K$O5WVA)R=W)r*I{FSmCP5#=ZAMuG!>=f%qUdkkuUEStc63-lLJQrZ8grz7AX{izr>`MX{zTE;!C4*r0ub)Tct!E)uHhh*6tOklh zUK+5hSKIb>x^AU<*bJaelmyM%-WC{XTJ7OI_^8V801bB7_B$Xs_nXVZN}4L^(8Dq5 zeZ2|~yjWPSa&8A(h#fL4F}O)W7fiEDnY?dk_*hf+b@<|kCl^@9B83CP7KWWLS)}^ab;4 zh$hc)9!BZYh6m1O?YHT8bK4~a>`i%%k@WH`88sY)9O>-6@kBPAq=B^Th|m27NP(H` zfG@c_$9>p)(dBKMkMKU&sa>;=PfMHjGj(`dp8R%cE3ex$-B>-hD9I8Zl)1QI<7!&k z`Zc>czpF;4>fu1;aOJ|9)9XBv-}Wm~cJ#x~ zaGyG?;_0yekUeV;hzhUBkrc~EAl{0y_FS365n zn+tM$+ni(>q6I5-Qqy$w2_}lue-ve_Qy6t@21T+97FeU0r72k zRuqiD#ujK}-7d;jK2(At7wq>F&_DPlP5-pDqlMV|Tehcad|?`+*Y66QF?*)$S8jFt z-AcV`6})xKXG8GP4JR3yfZE!wVXK`0Y~wl|!iCZsLEXylbSqd)^&8%B<#EM|wjgKm2rP?Zdgk_~%a8%f8qPAHgs79y0)IwhqF0vE6cE z;DMwArEIPO4`&(YIu0I77Ce>|lkDykvu)AlFlbr!c1GCdR^E^-es&1&R1z+yrLB=$ zot=yKeYhE5MRPM%`s}XdR_J1@XnNQRZ{$&I;DflfD#5y-aBsYk5wa>i1yO&ex-;OK zx?ywMaNxF$;lSmUotNJdTgiz=6t_y?5m_CGCJaYA72Gy`wHV+#`N}8BH+M0?w#5y3 zz@#oudp6j9CX<%2hD_gePKchpJJXylkqN8im* zI5LxPMg7e_{tQ<=_(iHlfb2@CVmIkSorw5K;Q$V9iLl{NH33GaS48nTQ6sGZ6?JHV zgdhDEy*vKW5%BDQ{q7-ZnO|*Qq6-C;43Q)O!^m?!<;bQx*J`++#b!2vGZ{b^sn1_lUa^&X!hrRcVYHC~ChH;~!B4PmnDN#Xs zQM$B_>kE*T%1?X6s_Da4 zVI7a8ell1#&$U5Vs2qeHb@Kfdk%5i99u&{&zRapj{_e6w&8*P)mF=dw`;&Qg*WpiI zexekz&;Mj#hcha)qN3=R3B!YdgHAy&Mjsm-tVUz)=4uz(>}VyEO!4ncOAGbo{r&w- zU8K_V8v=KOxbPqR-fKT77Auz>)3^6qKrKHUxXZrE=VXW@=EMWB0YlKzx*fnW0!1O# z9UFkW9BN5AG6?8TZUA*D)^-d@W(u()S02o;0-3=L^3(NA{eCh)77;o0ONcJ1XFxLV zMY|m>jOytBs0Q5czf1iIkbgx9_U76W4z5)Jx&00+?HZ`(<~}tRD0{_2-V%_1N&VRn zf58SIPJu#yRc5XMwoGK8YWNpgOF}dGKGQ;4d%^(-jW{HO2P>gaye2ha0z-p?R?KS{ z4DYfr_~kK1TM8d(JieI~L>YY6E5XL_I2O>Ia^?c%x8p!bH|ObT!#{+b8A3n6cc{g~ z<7%^qpoLqYd}P%=^_2k&P{CjVbTbJyYb-t19Dhj~LT;UAj-GE(t(N+Py9CVND;5%% zz^Ah=_HR@~=v7z9P2d>;a_Y_YDxe3CJ3EjYi86 zah!#e@gU4sI=YbXZU;o^)q3Iz{|ZDG`MF%+ayu>;lZ3Oh5~@T6tRFjlvg(`L-H|zL zkV;q7wglVDXQ_8+-=sZn+C3Hf)^d(|btv>K%&lyFd~>TJqqk*Pkg>ldZ6v|oHaa&P z6s2m5%LwS%2_Wqjnymz4qXS1}!ad>KT;7kaQ+qq5$jPrWKNe1pRar^wPcC?>@IQT@ zG1`A=vD~P8YdtTiY9rWXfBEX*z}?WtAr`IlAtCyWAuNdWTO%-edJt7Cy*xB!S878B zD50**B~Xso0Lsq-RCex922%_GBAKW?MQq=5K#cya)#=5HGu)U19bmc7lj>b1C+j}&hJki+o6p=sqa%Z8PW)%Vm+m= zXID+*9N9NlQ&MTuX+t?cFc>83bMLN_Q6U#NSa)hlvKZCMw1d;jt9MT`*k4}IK6Ut;9H+9VpCsbQ>NaqTh6*}~TVTC-|dl(-v-1;^WRkV|KIMQWNYmu=zKFm;i-VGlkE(a*05oV)VzCWyGx!1L<6 zpb+nNE)MvPIW+}Cw2>N)7ZyU*f>Z>*pB1P**NOk#)2Ht8lFB^6xvdq#p>_f=GaFWM z0~o?{pQ@(SOC}+Coxw4Ik}7*iBZk~xy)lxlW=t3!vJLwH4>I*srl@Sr7Vkym#SAD> zl^T!OFkRZXc68DjWmd8e-z6J=J9O;8ouvv9S?89pVzNa2#DW#N^3Y6}_DIFsj+m|s zR@#yFrlPE=ZHkJQ2|*8R*~w}7eyYM7T(-1#XG7u1GLvTxA$yJ*EW^Fqo;mQH$^x1tDzcOZXnI! z!{2aod@01|QZHN15$j?kuB5e3l(&=?kkh#ZV+?{DVeJe(XKe;L1a#h9QD+iZD*jCN z;0vQj!EFg1(G@C>W|16n5rvC3-22uka) zHZaUD$<<&uo0LHtIEAQW+Zr2I%2~^N@~f9&0Nz=q3?$GMZSLSEvBhp-i_@z6H*4qf zGKMyUu{p~*VH6i=it+R47(I`Hj{%$GtJ$ZC*~x|UFJtLEW2Aw#^RUGpLn{q%lYEz3 zWj=QQj;CJ0C4~;unKlZHJlQ@3HAQ?^daab_zrS&1+i^!i=l3I9$>EIO&H8*zQ>1l_ zQZ(jidjh1Pj8qapbTZ78iLSh_E#=pNqWE>8Xd<*Phhd~ZBJLTSIfp7YBC&^vRJzs@ zUkr{KdoU0>ju+Yby z_l{S)pn~P%hDsshM;&{9uzKnwXU{A(`?LT%3LbLQJij8LVl~~SNkIaZINA55SVrTq zGY6%@&f&JEvUWv?#tk}qEweKIJC`}A#B08U{`uuD3VGE{k0(8YlJcPoviP=pbZGP&DB6zF-Gy)+pB_Ish z0S_Qp!1HN<;%cm({vJ3B@Jm0CnuyQJi23Ib#P?nR+FEf%Z{5j8U-oyGmw<#91sgX&gZdjHD+eEM!vX8YX}=UFs{@fX>_tAb|Xz!6ZHeO(-cEJkx~R zzO4dq^Bw`|`(U6jszxRGuTD?^?1HC?eE~m|I*`UifJsLHdc!y9cVWgIL?>V+>0AbD zG;F9NOc!5kZts}$5Obg1{Z;5Fm7;TDv5;MH7vw(hMJ~}~bDn~MddF<$FB{6+oy9*H z;H<>l|NBS((G;|_CK;+K32 z)^wVB1wLAsp+a*G2T7w}9Wjeus765;>bT57i{c zi~yADUydP(%9_9<2w3B?p-U}n|MCH3PECwlwdc534i1Mue=eN!v75UZlqxa|kh|pX zJfpL%LQOP175gXT`$BYQK(XcJ?^E&kkc%7q@v@TJH{za}FeVwiL#3x#v?h>KVF`sa zhAjknBqSM`QBQxICP7=It{p-@;qodJ+N|p?q&@Py(Jne<>~8E;ZD@#oZ9@VB4l=Dx zV0L`*W9V24*pctSYvbmrty0ZS%p`d4G!d}1`E^6kKVO8T>>vU$N@^Zk3QLfBM8~aa z#NmY@piT$G?DMSsz~gO60Pk{xsYeiRi6~M(?G7B6g6GOOx}4^pt|0b4zyMcHK2+NY z_5I6z80ptgz>cjBBuJk)4v^dk*Pjf-GVKU5|1TodgFj!Ikkox*Hy@F|m0`(mXe?&b z2aF0o8Q!acsoYZZTVTAEC+LsyVE#0Mzl}%bkI_Nw;kap!zJv@)?*YJo`12S0H6X_S zHK0pB8UC9QDf}@&6Ovgk@ZFY|fLj4UB>Qi`MzL%F03pYFfEXV{)wN}!hdBbwO)0t- zd5q#nf8W9n%pmU;u(ES0P$s&1_OEO9U)QLrk91&yMjwNtFiK$jI|~f$#zHtockKMhFqw(?@{eIWVDaFmtG5XJach(x5x4QEkj&qiUJwq7gzFXC zI&BLCoc~Pnt?TJblH^)o*|+vHoCdPJ7l;e|Wb}-aUy+W-vJC1C^kPggD7r+cY-2Lt{489S&jL=E#l{fdWRa%vM3 z?RY=HlZH?e|APnXZ<9axlOg(<1CR;{6#|TDd*Ga40w$mF-%S3Wev7}){jcW~;0ywH z!u|P1|L#}(_c!{tLxO^KSpL`Ag9D@f$D?mTG88~u2=znaF&E9r{B36GwWTl#*)`7C zYORg$z6^fi*aZ`wO4opjSfQaw)eh5m5KGd~>-oW06N}o(ae?=xA+cT{P4U_|1=9j0 z9TEA-=L3ES4!e2va>TGgJM!-f3^QpB{MUaxNG9tTW(4#N9B^Gth>|4{dR&Y_>!p!t z*R>**ue`-9ye5D8d|;mZ{6t5vh`-%_!Xk8)Jh>9y`Quu=1>&sdu5fSO$ZGeXo(Gq( zcT3--oMZ9)6E&uJt`VlFo@x<&k@1Ne4I$~6A8;rQSn>C9z}Y<$zYl;AhJU#>8wbV( z>^?VvCE-Uz67P#o(q8~&Wr8c_fN|fRy+PXL9ulT&?PeE#mvo!2OMEps$?(~l@wqw| z*3Py`q;PR9c{6C}X$RuG*Z##}eEgM8uVa=jnUxoOy!2NUma|gmbNhYVU3RY*UQ2)1 z?lOvwn~s|-e-0oe@v7DT!@P;lKrr=ZCdKE!Q;mPojsJs}HyDJO4<}F3KXKpya)#ds zK>Y={9UOBW6b1-lMI{CM2GIa)uBS2lmf&{V(jB$41OsZ+1egY!1h&Zha?9?&T>oU? zB0@USO2OsMRIz!$@yqA+zu10_&Velbj7O#B2G<4Z)9=O$zizDNImYlLdHp!lRaIo9 zsQ9x@-IIn|QYE`UE}yfqKp?VH>Sa)Pe+T3w+2C$VdaFY{eRQNSk~_6eJVE%mk`|Wd z@YJ4|Me38E46sm>9o(lU8F-;+H0qMI#_A!hEB>p3`8rsdZ3(C)XhQVZM~?><={vismj*Xp3FK z5m2U)bU5{H2x!Pl8$qgo0tKJ)mLAHT9|~0ZvVRpQBv}VD+R*>Kb?%ZuoCIPiyG7FL zCxeZ|LlEWO*r6G|jw&3&y`5>hncha{0CFV$14tNRyveZBknZ7psLeNgoP}+;(Cnb7 zfAwijw8-B8v0tEA+|bzMXp=Fii5!RrU4&V#nb(b$i|-noveYiA$RKIFMHNIbb0pRY zbDJq^aGt3bdbSZ!1^3BAwBDVu@i9<|p(!nXr|}6IQ`fdsoeUz=`Tf}+>bgg@-~`{fmx{+024vx$(6IKN~!rC4)S*;sIAV4!Gi0*J@D`I z8d5~Xy31*JgpJlwH__F#O=|Dj2fiS#qW5oq5oEr6bLaSMSmshXefBWqab@U`$o&iw zJHPzg&|B~E{pJiXk{&PXq0L)xZ^zE6| zHDF#Dly(Uf&vMP>FY}C*?cbfmQcsLkSobVwHhwefS7PrXeA-^7U-CGuDQ9Qa zos#$&_#VZ~@&Lp|RRC5bIQ`mJFhmkIxw|8}v>xa)Paf(d;dZ{6TLT!n>w*zNCtxGN zqppK~1mInK$f{Ip9Gt?^qrq>1v~^s6ZXE~_o77pUR43DInvS(j9$gD<+-}(ndrc7QDd)O>T_-Zr7kKn(*xug>lxj z63{@fw&G4h90hl@(QjJ{*o5|a;n;HLwL-Wiu{{0;j}mQwap2lj6Hs#WK5|@#iVN+x z<(T@KTHEjPlfgU>_LG5R31n6+Zw)Pwkw3uMZH_%C8ayu#vzvSXBrzm?fv?;9EtmuG z3B=oFUqBC#9=WIZYe)L~>&R}vzozeizsSGl`3Mvr528yV4FCk*AXbFw7X6rcdkEE= z+W!IwJpPiz=l(U#r`+jMiv_$`=X*i{;Nkl(Lo0RELlo%0KH@_B@yUPY`S?fq;IG+a z!|Gq0y5_!ur(6&bkVL?HSqi}aPy7Yx{}<#xmW;ypj37sA=W9Ye{y!NCODlbb?5W6=hDOw+C=SeAGnsI(Gv*3eW|jugPBr0X zInA2tPUyF4!NFX`m9bL$g^asxXav2QD@?F0O%`H7vr&96(|*<4s1NMj_B~ zx z(H;l{*BZ?VB$>t?H9+$OWN-@BEp^!a0L3?~-H#5&SSh*_Az`HF7T`#uZ;yR82Mn$P zWe-r6FQzhUDOJU-aCh6!+yjVFD{7a)q!dJcwm6~{JAe_zZZ4OA4y5Ej!*uf*97&3; z@{PyA!3LPEAlMCH2`whl?*DD}{%)Qc!u#KQC(Homv462yfUIF7LEE}O3bnTgI@AHD z`fMF=nf+wo0rbM)zj`MK0}211?G;+SZ5!nN74-T=E&eYY4Dt^g>>sDWAE&~Hl?5CR(_{`}u#ZQNds8I<9xeOh3u>!tWD1>e~8`u3C1C)B2E%I%bslBjJZhx2Vt@ z_!+CQ9TXF-V9$XZHiuy)^IZfxz5?7pSDyu)uoHT$Kc; zQbsW<_*l5(?2X!zZOCRl?_hG>oa;euxwhq5ny@+coMHDUSSy`o^ zxkpt4zIby?y3E*($!(@q2HqE?n1N^F#MVbMNZ}4biae|?D@Rbu@vx-Y;7qfxYZgjL z?mbTBx$Fy^6C(ba8ro@hj)WD4M+|5~N<*x6JnnA*wghs-r82gEi2jc5+~xNzc)BAs zL7(iUWuuD`(S1a>$6Zi3C159EQFKnm}}&&b1-qouTC!F3xqojfp(h)Y*1 zEM$udAU=EophE`g+EQ@vXDA+Va+-0w4duL%xV#PQw&NX$e9T=C`92}-r8iCqb0Eh~ z*WAX$8m#_sOnr>o2cH5w!9NV2BO3vV#?V5l$T(_88#(gJG(q+sYfU&O_8Zev`uB^r z2JPUZ)4H~Y^lJvFW}3lgRw)aRt%wf*RI2G z>YIryz*!7KsjwDiyR4j=xl-O3+P!9U*r>KtI>_m_mvYZ0`G2DeFl8BHK^lmO@H zFEsp5MkQ5EfBgioE-@q8ulBE5B2FCu;j{I^-(qOFc#+}JH4|QQ&g|>tv{_7~w>cIs zbq*GA%gxrK+FY`@_>RNakf%!hi`W)BZ#9#y#r$UCW>}$mxwc=0Q*9;LSIlKYf-2mg z^<;kdOs3pi>4o2+_|p4Zu8e_Q`Y%8a(FyH(kk(L9LPvaJzV@s~#Gyf4 zT;jNr|=Y;%}ZlRiFX2C+bCe%*?{W=sIPS_GHe$hGx!v)pcOxLym2TV1k zTaShI2W-{g!~@e0EXSv>vX^`|8f4Yq>gyS#NL-jg+CsrxaPD`jqzU);fWqqgs#)Fb z*6G1Di`oc$a8hFZGk(X@gKVjS-;d*~-gk0dcw4N{hGBuEj1J@byyELVL{85gYvYDW8rB)t2}8UXD*{ z=LTd^7jY}oz7BM5-IS*6Q40?4QTydF6MMXN#Oopf_ zuh6;7$sV1FI%)S0{BNX;U;I!Mwb&Hok_7N z>x0e{x!HK$YbJLPLeRCGs-Q;a@`MH{wHq+-JtBKSq8Bg+rzTzY>pTi4&3ntN@<)0H zk4m{Ga{L%9^1a3&&REQ!m>uhf3Ztupb}{224oij!JQldp_iJJnDt6+ z@3I%9k5-vyMcjVwQIm4R(_DKvETL%$A-$-vY7r%nIlqT`fsg0?WKXt~``|aFDL5x^ z;OoM+1RJmGpwZjb{0Bzazh9hIDC$aaCtPr5yUB7Dxa24xzWTckkZ0VlQy) zHuk9=e2ic~vd!oSs$1{yn1e9bLRMPN)sn+}5FTaATi0qcyu6A(jkiAY19u0frpij6dB&o(Q}dBjPrtB62z4ieF+*cNo>?9Ehu3zZL~SXjiVbb3{FwNj{|J>&@&Qp) zG1mPeJCCovpLM*7S(VIaZG4Al~?t05?0|Ck>kECo4{z?Hmx1 zya?1(EXTsph%+mei!Ux1jck;i zpDKi1PR{FP^C?V8ygao~Z~7R$Lq@P{qhrI`A45JYzHoFlHj2$RdTq^DE}|im{5r{n z7$R%4C$nxKB!2m0y8oc5vqHgbhp-`a~>gLblktaiujo7&~iIpPhzVse9Bg{v%Oh@Ll9;dXb9 z*dumbML)c{u+PKZrf7}N*TO@+y|*{IvofkP7GIlPlPg=_aws1`3CUv(m``+{Yu>`4 zH*$iS)83M>u~Kkud=|42{Rf6X>&uwUNXak?+&eMKdlmLgi=~Z;Pxr*USzF41jlu1c zvs4+P@k}Oy&(GlG%JgMIET0;|{(WOZ4J9a#G=2x>?I2Coo{_J|(ifACF?m-{X>OLX(3Yz(({x^B3#UAj>1n}ef)N$_Xv zb-yVa#3IgY&_SVt7#_cZhaStjPcL^SGkOZyJzvXxru54jBbu56cYn2K3-^pvu;vlr^i zyuMhf@v$7?Juq3Fg4+)-E2x% z3zH}}MITcdU2JSlwo>-H2o!YQs{YDm8UJ@F>fHE49{ay-Q73LRHTo<|=i-Xt53Pq` zYMDgiOFM`OC|(@Ns>TV?b^yTsd2@5o?Oxd?5ytQKm{W(67r%JjTy%@<_QdcycZyXt zqH#%<7=eDUEY|4Ltmxxa>UBNZ6h}sodEvF z%T+O0`LyGd;J2fmx1U}WzH+2fTWEaaNs_Lgqsfo9I(1t+0_*H>=qW-4vakeT|76}$ zcHH_B=-=P?gm%NWC~;jsWeb)V)8n#~lKWa?SY#rTue@6p|1NLR!$NpRuBtgTibv(9 z=7rOz)#|s5%w9KTW=+`YkdAZh?eD)s1ZNwRD@Btf{ev?b%$+Q6d=Q-{D3S^vM_aco zTnb9Rm=!JTeH6&a?`pLIb(+lW&N8fv20-nIzW6!dL+=PH|6U-29a_4YDD)=0PxGa#?iv@gVHVob) zjsqHu3yy*DtZHok@>662aZ6|O(Z5tpEmhj|;djxmiVs7?R+i(h z+5C8_5wbQ*(PP8Tuvlp?eXUlQ@Fd;n5 zAb=s1z&lA*BnTFlTR+4Wej0QW+N!$?>39i>3yfUvp9ziwwlV&HB|WBmIRiN}~=W9p62 zbx3k*!)DIF+BDl>N8EFuHMP=rEIZGMt8owf*{_A(u*`iu={T+-ZwkQsmBN3ZOnKuTvri*QJ9`RRJFeX;d5;SWgsVpH6p$oV;0D6B+%T>|G?CiNc@@Aq{2_sBU8wu}H zmA?}gfj0BI)leeCuIcwVfSxF~HjY}t@OVm{#2@$3!0ns^D3mis-ZXT4s}2zkKbW1(IT?0syw_Da$GIn9A}+fDTnR z8n~o8y7h>0z@j}DaMrxES3rR9@P}leN3k2Y zElnJ=(-lcqf>uklm5|>U1#9Q;SBvg=7K&6vVOFuxR8L|pl33kA=OXmQ1bY-a^&!xE zB5rx-H3ozZ^8`S6_uQfxd~?R9L$h%=_BW3*CR$W!>7g7Dg;)tc=9ribwQ2s8_>-rN zY*;n<$%@NCW0RL;AGzkvp%EIiD^)*|BP%)8F3f&?+&)YAkTPLOvFPe{*9)#nwoYvF zH#7{&-NCSWDXzDsSWimltPGb*aO^)WXGlN*?;dG4zE* z$aN|cxj7%uNo>7zMQiO@2oTv)PZr*FezKEM{Je+jmgbR`B_pM=p~R1gjM6oMj0>t} zYtJ<_E9G{OI-5M5_H_P5&W7Dg=dq+rZc`)cM7;`9%23FzssWRa4rc{N>%fa)K&Y0a zzY*Lww&4qS8k+4`9KQPJ?O>pP0ir-3l1DtIy>nYId{zYWzOd(1VSkivlpk^hxn#>k zu^p#n;6Y(h%GOp8i={EXd+!i)XxiOf)vqnZ_|Q`l<0f(D;7o4aEiGz7=Q=m4)jXmu zCWexijPQ{K+gWW?!xpi0*sA=#j?6b*S75o{jw_vv)H&E8X*W}(I{GhK#;f7-#;1;K zovs2(Sm?VU$Cb>IcR%!m7UClu*=jOgj3MO|YXMrCSl0BJ$MnipMQv4lX6TA6hWXc~ z(C4`F`R@hy|35pL*C76IG)JfWayb8|Yq|^`6C-7cupAn+J3Y^hrnllw!VkL{lCkVYn*h8&2YiwlNlWbxAZJq@wMIWqGwKL4nhDY1; zo@yLSwC57A3#zQeZ8smtsEo0u>06hS&0X9b+#DOp->+%vgZp2ZC5(vWhnm;ziy?2V z@Cghx-P)FjcyAZ7V22p5IB)ZIB`a?ZTy8Dyzl#D46bW+=Xm1b>3B0f&1F{<-jpRp^ zjCeSF1;H$!kebaOaeP|beM*)RDBrg+FWMP*+1cb(#3z-OO3EGhE#m&?gz%QD)ntI= zae}XuXor9H4b2?rDzU(Kx^~;+k&;51ii_U7w^b)sKF^fet?Mbqq+zN;s-%0t@urY8 zaP?wp%XXylXj)B-b)9pk(TK%(4KjgeKS^v-bGkYug=9u|Hw#g!sGHab$ty$+R#Vl1 zvkp35;}(0pQ)~_#1>iJvQMl*_po0Xj7~W>|V>wK>z2#KRpo&HDihM#~VVt93JadfL zm-q3C*KWLwyFHBDW;QN#u^SQLC>9dyrP-Dt*>ur9nH4tQztBtQlf4# zVdkML_;t;TT~Wh&IPbuy!PI?IU&LvDpjme37t|5TX|Dj9m$_fjDzM!-VvJ?v zUBhEyaU7xYFb;JB&^g2pfkz!D8{Z*hXN2Ipnwa<^OC`gKIS{qF#a3VK)2-x9AQEMn z;f5mO?WgxFu^ph7wx?;8ZkupBywoLUpz9Kf!^+9_5>g#KDAjbsUoheZd;B~3ER9j( zqO`;9-|cS997#5QNmv5eo~o&QQ1Cn9tW5EeD(rr9&6=J23Wv0aEQg7>?iI_gAFRGx=2gly zXboub{%Bt}ntFNcZT5Q*cZ&#Yu)nS4#2{VZT5Yi%F^Mv}fs5JLjj^yFmHjRoh~kZJgjJ=9v^Hsyz(mI=TnF-^$~nBp1qf`=BhvO--G>i= zC!E{HR>g*U&7{?W_UP1kLDa!}6Z{WC{U?#79bBS`!6JvWj*_v!c>x)?TR+OrFz zebbvP{hj5iH7lQYl-&h8rPS&9JJ*bv^T4^JcD{O8(zn#K2HmhzsQY`MB+^H`$Rt%gGxhT>G66)m zFU13ACw8Eo;_C?LvFIkD*(?PkqUs3g8EW;_>i$rBN4eYa6sV}x`Nxk1>y;5f^>NQD zK^1MSJpE@s^c=ZnWb(dm$hxU=;YEi2=E}{Y>8>5$+QP3upSs~IqR|Zt_x)S;NAe3& z-FwS-)p}Sm_wdiBLP{62lu1S8e2O}`Q%_+Q90ByR!M=k9-yKEo6NhZe27avicdkmP z3A@$Sc;7uso3DCu+^xmp@-XNu2AU8pV*^kRz)z`CGs;_$nB!ZL>vAvA8EZvoOKLx{&F>Xfw+aclbac7D(Y_bEx4L3OxlMhpx zob`CN%fj7Z*-~DVQ840A-CUi$vavB6*Ho=&uFK?o&a0+$^G9}Y5!II5L4;V*-ck+m z8yug4Kw&Cga~sMgL*+Gkw>R~S4Wiw+AEYxwhc2J94os9MIXxFuQTMOCBjv!|W*wYH z(bMrAlakpVO0AjF!lv$fdznFVaHh$9fPzevptt6hRj0GRQB2B~Mo6-{Y(wI~)qswd zwA9Xb(a9BltIcnuw06?Q`Zt3O^!Y*S;x3AXL$_e_b@D)AZV+Utas^Wz`Dqe$ z2GES0G>f{j4OjQ$itg)p>9KBZRg$DA12$ES>{&xT(wfkiC~1$s&SXg*YQvxlI}%`OQNLy;W9 zCC+g!AOi3YnY2pa_)b-5CQ6gHT9GIdCGMn60*J3f?t*Cv9Pie-S) zl|FlMaC-6g`mEcC=r_IRL`(UJ+`e$Xki)S@Uyi>wSRiY2&w&aykm!;gdG^UDDNaso zrF6mVrZ#HP%b$|C_s;ccFdsgX1%_Q;Afm9gLS)Qbe6ki>+k3SV{=@i#%2=jO%je5H!fyczP1Lok($w! zjzV0+Jqiz>BxiK!pn_FFdbOUioX3^n3(vGzL{@Cohc(4lJh-(5qIe989RnVgJoVNV zddlSOQjvmJ1>3NacyD(eDL!?>|N)H`d_Ef;C8>8havwt|CAQbZZ+Z*V;n4AjBLiy1>JR_uXU zc&~XC_V~G^Kan@eQa862H(gIYK-FqUi@WDZndQI@$Weqb(s_WC@5GFX+Q*zlvzUFV zf<2$SP5)XM({>m#9L4r*cH53aP)zS@6;oEvv5*cc&OI5alf*ZofcQRJG&XPHgl4L~ z_HafL$*of=$#B8#<94a#*y#6Ja(tFk@$Jl{k=3LDtLc$dw3Q7K!iZF#Xj&shk-;T! z^C4N#dT_6N-;QPrG-*GdijjE#_jSRlr(PHS&wRc|7f2NQ3P zVY5$;p)r8^QLnNtO{>1LS{vU|EkAITmSmdj(DmYtnk{cj&?5E593*-au&@g~(pCb8 zj38#O0O_M&emT8aFrn}&%w2v^s71c^T%fwzfcKZgEo3iu%J}^!mr3;ok=5?py`Bdo z!?^rQH4`Z!io?$150>B+SsTYr*5)Orq=_aw⋙iyf<{r_(*KN&`B|E!IZRy(GnZ? za&mSr!FVe7IWSCJ~RT_!;bR!uKPf2hq`dYvi-SO|ADa=%upnDLhB4VKR+ zomlhF4v8B|OWiIjZDdINsLk~m^kxs$4~peGGx8n?u{N&EHX9pPk9Dd55L-CfB3S#>srzmMcv%;)u#EY=6`Sz>@&f^?vm4WWO`{ zb_L{f{G+n<;b_Vo$s7`X`ZSKCvF^DnMOd3HWMA}iXm0!I1Zn*Glbxo1t_`AhESuYj ze8nQe)K_{Y&cUXe#qzn_x~73}Wfei)+gzci5~_1DruD{mbd;LyP@)v8X4a;&vau!7 z77Yv{L6i{aPGcvIof<;CK=P!OsPGn%)8^#gv@jFc)GpT`JF|I*Y9Qy@2Paqg{7TEt z9XJfy*Cw1fC&)cNG4pg}<1CUFisV!|LU`PdUy80#1k|p_)Fd<7Z};EJuqDbScwf~u zkD285i=LL|>b_-@lGDXg4Hr~y4XW3wv*OTG{0N!KN|DoN58RFM(9Z8p*{T?WP8YTh z6b;2GIHZ|0rb*P*nUt@G$hqO;q<7jCELAL1k&=~+zZhAPv0a$Ho}QlS(x#ib+Z;V;0Rm3^f{BE-Hel)=+cg4)9w7% zx)<6%P%E~0wDNn3&O8hCc>j~(2`*YGvJLR$K_Bf0(>_@jJ+MAd8|+`$iYPelSD|B?d%+9Wi8~^Ym}$ z$!&O#b`(c#B!5c`9(nNx4j`%V-nuP+8+)z`&0uyk@nf@AMI$MW*3xpB-kUxaN#(}t zf+B(p68svi%H4y*fmYX*IV-8m(sx)sS~1^rclhWaef(9~^#rlY=VzfL@Lq7Vtq*v; z3|`58Mf7!MzKD8#QVvNrr&UV_?#JkuYRfsxr%Y(uH0JN9 z0V2`$e8S``rtKvh^bW?~)}R?7v3J7HtI%@dB|i9zeY{fzH%R77(%lQkABW;eoW-Fx zL%)K~Ae@7_In}Pf#M)Y@NJYRk=k$JqoW+5oYRW?od+x=Hb%|5+!VIc{3uSaq_i}f?3XgkGwdonQ3K}n+ zQx5%x{H;lDPDC~s$=Zu&HWqL2`;;6~7QG(J^p?l{LDfYgVlKP=wFmQNT}u|M?DNC3 zS$0c`3OBEToeo~_#bymPFMQb?3NSPq=%hzn;7Y|mA^0YRuKL$zHs)_Plhz82-;j|M zek#idI*zG5`#M3PZKR%1W=c{rA>~7lHTg%KrL|wK9Tv?9{oOSj`&*Gz8+d#a6wio% zJVCt%0f`;tT65Gwpyw0odYM91zIrX%AzA{<4JOsDCC~cDR+Mzp`K!J>dgZg|3GF~s zy7p5)P5_*J00lTjFCs84@0+!&vD1HaI4VZ7=;!lkwK#2GyEs~9soi6~z8$sp@8w>BK^f^+KK!R=@Qtjj)MEsLa9>vDl_0b58a=6d1nb4M+h{cU+}-@Ic1ipK zF7-#E^+Z!|3VnU-tq*N$IbzXf5A?7_bVUdUI5(vFU$*&vk)E=)gIR_Mf*-~ZZ6?qKJAtGd*u!(P8yjNbM_U9=#&YFf2SEx z_{b&O*^bHSfemT__ZV$2-sxg5&MR9B@odk4Domxu%Ny}*rd6Ib88Z4jW;1e?7fKFF zw7bS7{00P)cC`f>1riU%MT(r+JyvJcSMIVubIe%7RAFDhjdS7mhWxZV(7a%L`-)Hb zWUq91+~6Sh@A3-|WL<$)@ula#R!PoiGWfLIphd}II9x1ki#>?U__2s4O=#Zaa(Zg> zdnnO-l}`X2=JgUnpM9S)l6_l1Ywi^nH&j@_tOoyzHoJ!^XeGM4MbcRUr9PsLR#=I8g8w9jiA^NV)N>ruWkZCGPE4944LFdc|X(DZz4hcb=h9x;rcT-eqi6klGp z3t^Y;th^A&Dz(!d#S_7GRl2BcvrBFTQhPOw!1&;vTQQ~rL zVtmDGYU;=NN_CbG0k59wGcaZ!M=(OhahiyuM3s=X?S8SvJ9pT7jP-=(5@3iH(E(`PEP$GAy2c0tDPs}V)iB>oMe-E zB03xSsK}zX!W%%IuM$mV;UeF}GvK!Q*T7^^cLZ^w ze+14q&DUaOqkGp)ekJ$(OLsSwx+elpzhCy$0Rp&CT=?Oo`-L0)k@!o~hlXjJGh3Y4 zLjh0;e?@vO0C3Hm095_1t#iTg|>#i{MgVPnzYksCP_Dd28FVU}w zvgvBcbB$sl(*5Szk6*i7Z#T^7!<~{BaK6fv$azHvJ1Z5IzSYjmwFy_9nKHJmL#K76 zTCo7hjnF4$^O9F*PAICzLGMx|VcQyvboPYk2*fI*6o|awVZ26RF>us& znFw$pdHb8Ce9S7FijIu^-u+=Q;z3@1`9T3?uXdtYet3`w5JI0%%MdZ`ZpIIYdvS|t zM9Xe5$T^r-lZ8`*MEii|QN!sD&a39`qsmSXw3436&kK8`my!`PV{JAHZJ=?bI;_EY zSobt9RCjbEkH`@(OBpXi8e|$Ay^(n7**Q;zc`lwKZ>5j$oInqy58snmr#3g%1&Tda z4MJo?KY*%*;_c&X!4cWQNeV|&((a^$F@6iHMwSD8BY0+=j2@H_S zyUrS-D^q#N16Q5N8!t?@PF~W{F=|M5U$aSH-BWtXL>2Af2c*1pyI|PL3iV zBE1O+QIQ%TA|RcpsB};ekQx;bsgd3Z9T5TPy(B?;Nhl#e65rNS@45H=-+k+Sxc9?5 zYn^qJLw07*o|!#+%5UBnMqO9qpq$f{p-2g5QTK-O2@?8)!u{kwI4ES~%I;WYIg6!y z%T1FUQ$54#HH7qt4OVFyFi9iCH!8%=KFiOSw%`;dTupU(EYu^=W6^d2BKUa!zT;;v ze>%kw_&73#ljeRrmIBX?R6MTvpeO5W-a}`XT+&Lh|3H-LIsK<8(=p%a#9PD2c2kpe zkjEG<87RAhk8T5qaF9uJpCk}z_#tz> z(i$(p4izgSn~t@^KTMd7536tKeOA@tdnbfH2TNWyPa+Xy26nVbNYzT3WnroAObo-IW92m6$jxZrgt4tjU0Cba!FIwOCxcIUw+MfYk6S#Vc+h;pI7 znjMiz5iB9ZRf~aU828sQ+&7)k0iBnhr5nF+u{p)tG@Ys`6a@lC`=PeTrz#wS#W03> zB>jfJaI+q_<=qj{c3I2oJl3CI6-BpAx;zZ*lRDm#SCA`eth69G1xnxPCUS^dt!+L^I+~m~ z+n;Z2yJ=!R2RCdz@4=eg_xUZKaaCoZzY0Vp)xUEUTUL)5E}Ft!t3BCUdu$HwnsLiM z{@@7d3wIKC&_nK03 zY4KIlZ-$*O&OAm<9Pfme5@3Pum>{d;JsXTnEt6W^?iM$fa-F6)A_q7I9o65g?ftMX zar?@8qZl?SIF-Agx|dt4FSlAhL(O%6DecBr+0i?nKh~Y1x^AF9zPKGALkZf5Av{L~ zl`rwpGe>2GMhY>dx%J+xy}2~L;nJ~`@^V+>SvA5i=;|=X@E~ndpm@8m;;s{_I0UyBh{Y?h3|lOw zvi7AFRYwolnnn-kOG~Mp7wqOa|FXcu^0QHl)k}5w+&Q8NoIztC&1`*L$FqCF_=KUe zjNO-LZ5i82o{R1GJDGSYrQF}2td5)LzjZ;Ni;|`xLH*q&p@v1f5|9&ia49cT)nxN4}h|I|M$}#D4Kk9Wmb%E45 zYfy=DcU)NO;<2zCHq+$id(F}817~dHRbKpt3NO=d+*Kn_euOQgj8|gI$5AH(Od{KAPV`X*n^w-`Gz{NzJ70O+Lr|`M z-7@Dv*~DNsd}yWi48@ZSBkDHhfLR+}iCQ;TPRuJs@EeqQyp$>lOuth;=DTA^*|)0% zVzw#Rat~)N1nbz%F{PLMZMm?z(KI*iz4_SZoY$sJUxn@7N8)kJyF1X9ns$=3NWfu> zW9r-{eZi*)$X?r3Dtw2zRhQAxOS681=*OFMcmAA-X1z(V@?2Hz%?}IPCbb$f0enOw zuU>_e^XmuHwJ+DYXtk^7=C8ib#a6R}zLr1?gWtW8_bINpVHwfm$}ghf&ITmucBdEO z-)t6Mf7;0JoF%o9k(r}Py)d|5GuMi@z*lsv5!3YRk5DcGlxbX*F{MYkTfP0mq+i~t z{mA#)qoHq!>T8Epcowf0UHLK)Nq1r&Y5;TdOKpgfeA7)I?-tM3xSYXQxw3Qp21v%9 zs)(he?ptJvL}5oblp}?np_!;cQAay{7p^Mzhk5E-KtZ7 z$B7N}nozEhg;oVMbFzisylSxx@P)&n~lfMd$rz4K!%P(O6(^m_os*Dz6?G&M;{HVwoquIN&aiy?DS zS0LN6<(QrCbQ07v^u*Q3$u;9Un8h*V#(V#$Q35&+;{=b>jWQ|tfiir1w2Y=v0qLpD zScHkthUsY+j||e9cahYymFR`=80x$aB!zf@_MWQ05k)y+^$>-XjgUps2S0+44>S6k z(|5px0cd~#aypufr?s30Q-@v*4rp!V5GT$9Jhx>8QQ|9A7qqTUYiLlIH=!rR(9=-H zs|nk+pO|Tiu)_m)nkC;O2a};YV(=BE z8;Bun05TeS9?OgyEOToFqd1DBA%!~W?V!Eh(E+~>x^|%jqO09X_WmCI08^0LaR)&= z;)BE=FOk?i>M*&ZSF#RSQUkY(1Q&A16}b^Qf=LQ^kw}&g1;yT>upJ}a-fDS30xq8F zgPQI_u<9x3!@fCC-C0?lQtaPi`cZjCzie~kie8Z$iTh{`ZD6}5{hQ27*mI~UiKoLA zVwtC@+J03;dqDZrk7|WG;7=QoBeid+iOA_kUK+IeHcAJwnui%+A^w;jVs#EVe;oQ) zGK_Zd29i7rhBsauT;3xQLlA8_!IYDJ3s0jaZV!@FYCNdfG;TCyqyG_fYvKA1KM8me z(TDL8yZdi}0^uT<9PJZi&0`X{Q%mB|4Wro2SD0yIuq_76Mx6IhhuaO{l_WxU44_5@ zvFH%wRGzFDb5sD&2jQ8*zCz`uxJf+Bndw9=IKU@m$hOO51fb14o)%t${zN`X({Qgv z#z6J(nC%n&+X41Tw8NqgddOY4yJc#nJ(vyA&%1g&gH0HM=01SmhG|;e*#tjT{JiTR zCxhmNlFjqLnF8>(3GTx7`Oz#Oc>&-A=KKI^z&!1m!`$o7L0lH+`QuNI1B202Y9gvSVouw8!?;J_XpdBClbe%Wwp^3c5N z(7Y8B(Im$1@cB5BxnNDTWVGhphz(Z@&dX?4i(fE)YxxyXxuML5gdJaP^viD4w7L>&GiWLKT3L18axrZYu_Gs9w+3c57H3;N~6&9?gI z{qGokqj8m>N0AAW(B%{?$+-gype$smjc0(t;x~08G~#SgL`1{n$cU5d014zqTS5mreu9Ee&4IYN~=8%OfP~Bl_x8I{`Rf-qj{;Ms)A*nXIcaUudLE`q0d(#tJ zTVjVrM@(?^m56?Da^F>q_g)clE@*ns$u2;>28r;ET83i%wVDAErHm-AP^YD;9=wV( zdPGQX?Y!rUPvo&*u=-H~?r8cNvK#(rpABLUIf=1#vIy*NllIo1f!r8W*h$|sweIA> z#S-f+!xek_;NW?g>)MJA6_f=0bFBni`SVEO13@X*T8`dqF)*#_z1sC2iOs6zPJe9C zG67+bJ2kdJ6|KWKAWqIhj|E&Dbp$hu;n`;I!}c@ERIRAJ(7t~ib)QogCmMx#OcN4A zkxgj=-|1>|ztfSa_70N_)t2wy4^j5vZY%gNLTOM~|4ng~sbpnc2WkPhK&~VA{*sg z7rtxSZ8BK~t*;8X(n^$SIPZU84us1oQy8)?u4;X^i;f&y(dlWnhFtwlx80TvlFJAD zv=OIl$VlAfKorIfv<5AHbGy;?sNnT5UN8mQ8&gwLBkE2gZpd3Hh7fMb-Qkbxa}*gQ zxdADQE9td`1sy#&upaP*1>ryH6Ogu!?&IYrO^~7Q?H?W+^V1|GUu<+{b&2q~n0st5 zGv7Om3KDBc#(Y`>snC7IOJA(H^Pu3m!7RWP@Kcyj@q1$vbVMnL!2l4rX&;hYSZFzm z%riM%g~rxsQi&sslCYkjW7u@_oQ~EW;|he&4>A@fM+ZsqWM85$$+5PZB83;Dx9Xej zbF#6pP>i~$e)YA~C1twl!Ki?y_K2e`ND>c>db$LC_9Iyt7e+sZVy0Znl;;6x8N?kD zdRVGKq3dZeGmA<|`h@+OqYq}p8CxkCZ{M7Id}W(6XhNOW*K|Fq$65vJfqf1l&r%|C zq0xEQZ+*r*=CC@i!%W zrwix;SSS69sOJ;n6=86%qfO;=DsUa>nzH{h6{V@y*z7xC0L%S>L$ ztgqRyiFze(s#j~cD-C14VTN%*2ii}{Hgz33xUqX52>iAo?T9ilumQtl(a;^&HTVJx@P$8>PlY_ti(Yv^l zd8X~x#dX_YYeF8@jJ7?SU)^{@RX?>}kv|x-d>q+z4YOLHF>>ZRUFu8ZD_7PSTgMg2E@i9GYZ36cIRj$~Iphk89b)hO=|enFLZWW;PwRHI(-IR?SQrXFM{x z%-IA;RR##6NXiXq_VYXjONrOKDq1GV^^v_ zcGbQMvv*<^pGa+aFLJ+r@lSUCK?|q_ zlf?WsUGmgUKV%l>!@99}J)rnso&s#xt805*f^q=hZu>h%t%IcliXnNzf3WsfAwaGc z=or+I9>9pCAm{LAk~{dvw3OJDv%v&Qz|#50JwWY-`j1%KTj za6KmPX4#a3*ZXhW_-*ZBTJcgJ@(;%T++PrFmrX*n1-k9_E5+xHe{4^nLz>`5zMJ&( zFriyR8NYK`ukcUCAuZAp`gjAT8P;-BaBlWn?xijyhHMT+Lv{+(_Qa$5Jd6ImGt zuX!%SPj39AH2(MF(fTVpL>JwWDgJY?mZ+9LsnB2=GT^W~@VJw?M(qZZP_lQn;VaXJ ziK>xjz|a?g$!u1)iHM4~@~fx}g>sT@qh=d#>ADV&z8aG$*H_T$^x{7`6>5nM4<5Gc zlmtYh#4g@qBfz-NxDws#KjDsB;&Uhf4$D{K zWqm+Ts#KrSVFFUGx@UzkZ}ZCl>6p%&lVwt0UV1;`o08<8sJ#cqh48koRP>4=-n&@& zmaNab*eNR26z&`RiYh(UT5k(!knxxLPN#JT&33sBz3Dk88a)L$aJOJBgWcq2h+13v zX!srN3}36ZwkiOO494C%HJ&IMk+bjkAmT85DRwL!y?E~~NNN=jCaHApD9Qm{LK;XA z18;k7Upi>y6hS_7(T9zq^cud?b(K@wDN%I+B^KM06JMZY6BTYS7M3Xn*nun>EAp>v zuhM4D3R#$By6j2BxKtrV01i1f3z!QOD0gw{yk%BI5ZzkE~8>_$S zQJ|yvEr~VeJQJGdRuS{x2m#p^KhRA23$HMcb3=;ggHV;X&p_{|>u&ru{Xh;sx8iu-jjdzLAwjTi_YfS7^j?VC-Y@1XQziSj`iO3 z^pZ-~qHZ^U?(8x6BN*?_$On|LKv#$G$@aLl@WAi zEd&()(291?IQC8qgZN=x-!j@yHT{~FOJqZf{h351JBRVgyN5ZaP}nT@uEz?nh~-)) z%8v!ROVGz1;seg~%iOVzzO|h6#n*m3a^P%1b3|3ax*e}bMpXN&^qS}`3+=VMcS#RM zjXe;TAbKrp?-!Jfw|q;Zb&EW8zi!|6yGQ^&opI{+En;Jbc>DN!{ig)?Kds1 z=F>X!TkhA*oG^Wg&NjckQ|3b5P;mx$UN%_F&K?IS`3MTDK@o-i5dEXqK(0_Ft99OL z7{~l1fU}QwlAP3(%1HEB4Hh6+Um3}$A;pp)7Z0P_ErOX}H-D|Y%)s{O)C)yY;_4D| zYLeN{vVb5kc^s_~QsVw}_}SO4A#AjLqbgfds*CKg zB~^aw7U(_j_zNPyS$Z(V!ek&Sv144?uQq_w>vt*tsU`tDkoBK34{fM@M37jDmst%& z%XG15;Vf?6>9^}SRZO5g7BBKV|CP7s6ZZPMt9dUtN^d@)6Atas9qJjpN{os+ucU!% z=>|BlL*|YRRQEOkc$LL-%k+ku3R=aFKhDl>Ywk--u;siG6LsFgGA$s>!G1C4+DZD0 zg+4j+xp|ZRehKRiWE(8VApdT8&XTRMh0s6#>Y;KbP`hkkg}-}6LcF*ivEiM++%w_8 zzArAqvR}uo&e~`&!>5nnWu~TSV7?U!dzSBABFTmqW|!TXB&z=_`{VT&`@@F(!!(5K z^dn}Gz8Ant{N|s(qW_P!rR;3l0tak+3sQju-bdLwrsg&B9}U!5AO&}T6wuB8lmd%a z`kM?Al-;N?%qNvUY$)IVjSXf0VMFn>Uw*Zj2D#l7&M4XmDD%5{S#urXmtxO9<>U4v zLa#zZMa4Cg0y|tx6^TZ(*!&6*9*?q;JvMC?Aa%a!PJ#64v)4~(%Ymkel}JkF#b*5$ zgPiN>uU^IItX7aH1>3SGgts_CuSXM*nc4tXo_o6wR3j;p%RdYzTNgS92=BnvbpQ;; zFlczaHS-v>yZ*;oU4ct?n;S_zSn~wHiUY?!MvVODj{vCayMO=RzhN>j=KK$H8jF#^ zHKsFur~lU4`nOis%nT#|YQ}5*2Gp!nL~4Q(ZANJgJ9f0VW-F1B4P+}M=z*U;+4_S$ zsYn&>v&`0<^I3``_1wv8jmSc?UxZ8Dx~^|eAP$o&>nT-T`^azYU^OJTO=S>t=~L8m z00}b)vL<>ePis%zU$JbEOHPw)F-)_P`KDsld7;|okiejn>42D`b4J%R5+a9?Zk{zN zC^tWmBCD-V)G=|?)qszxCor;t_?rd=+{;&V!$D&+0DKq*52FSs4z_c`4@7vMbkROz zijg?}`*&S?@eX-A>EBD()#$>4|v z)W*fPAR`J^UAs)nwu5dQH3VZ{;s+jSYXbzU2KSce5aDMqlcqqyHk(5A}*;9j!Jx9pu24a8;`Kvf4CxFw>O_0PkQnk z84P9gpq`mnn?UZFH}`n;?si&d+;F=}NA81gri06w`M;n6mjJslfZc$V2T*|BLww*` zeJk98B7=15QZ{r@oRO5;JtHjvM{=Dcdn1zyU_)c7pYfZHiHiiS%6ON#EoK&4{|N8k zvhEu)M)T`xE(P*-BtlX;%K$3BEMOMUlcf5pJz<7_c151Y8mdiwFP)@2GDu@->D^Q# zP;<3ytnY?!Ry^r<>c~C(WcLA`!H|-_1+y_MJJTpC<~q$^>R#X)18W27?=lzI?AFXPxA14KM6utxU{k!Ij#Y6 z^0H%CwwKSfL4(2zpJ9%xHU|Xg8K2XKk|qO@+yI|0h!jj?}I(YCiJ(JKcQowSU#c&uS8lms&KdA^4E=Q+wAOnqT7|w>fnihRR(r1_q|Yo|;3WvB%4b^shVHAjtfg z@w455f%RP(@kI|!oR{1Ci-stIyLx7ed~_9Gm0+HDXPJ1@iUFrhi6fS|ETUtj+t@f^ zca88`GaZ~!vga%xA|g?&^f6KK#$!KYf9CMK^Ul)4oA3`SEANio(l{r3!i8i1xshWR zI9r`tzhM5R*cudmnqVs>r@7tI3iE9-b>~v@_x*x$j}(t z@$YoNaD0uy>rzf4Nk+(jG`LaZ;IVVx=~P7^;Kf@w2^H9xJ#MM*w;&d0bZRAN;|Zh-eiU`0RwnAvdcT;1qcB2g}; z*@d~@#w%RguQTZU9kJ~CvXv@<80WG=EsJ5k=Z5tO-~4Pu^cwoAq9;yhI+i%+2vm20 zx4@3m}zi(k<9KL%zIJbnY9N&W+a zC;pr=@!vLgB5VHVRKuQ4WWvueFZ_^JzKb8po>*;F^S*TVqK{!6F?UzI6-C~xd}*Rn z*7nM4U3SLh44Z4|9+`VcT6*wuV!>d00XnjV6GiDAZhi>26qNqnxS-=e*3%mf%G#Xb?3vxr;3pfV{O`#Oq@?`x`S}|}m z3<+r3Ga$M3vB>Q^NoIg;Je4@Q9x05Z^!~J01#L1KoK{Ya$yxkpw!nT?-SoJmqT7%peczFpCMr8curYnn z#ud1YnShmcI+<$BC(8$lO3oXhs8!6gt4;s(9l{hDJK$-~{rRO^YEE|cGhQ##?gq^o zqz48*IZRIw;OmnP&MGg=#>)$b&i~3E|r;+1Co3Z*fhK}veB5_ zOMBLJ7|{87_B&m8Kl)%m2j3YYGVnH$t$8p=N#@POwYx>NgPZmnrkKysDz-gRgzNrZlzwDR&;QTp-K1J2UQb zo4c{7`y|cEr83coXczHT+c;I^x`bUjGfl(#&A%fvMyZ#=yxc{{3ok&q_*3O?Wfw|G z=rOrkLWm*(Y(V<6q2!C`?{w$E&4s(@`ec#=h&~5$O&Gif8O`v}9d;#Q;LVGoXi^&g zbf)jU2DGd2GludJ3WO+#dJsxE2%&_G{uBKJnNN}M zuumSN76?}RhYJ2F8|^L<8@nxuigtI|7_#Sbk45jflg!dnJ;>JOXP_9s1HbBVGF)Bp zcu%#<=^Y`mwdr{@g9>aZvrSY-D$A?5*g?Jxsvfz7ZV5OaDV~%UG8=WV`j)jVI=fq# zjB;5X8S6H%0)-tLzd8krl$%Y4Bo7Rk+Umem$h!d$o1{4Pe?j)Hqs zbTahDce-T##Jyk4xif^2ooBnYoA=l{QzZM`soXf^bA*iX35SP1CFxtCznp$!)}2ld zT?(AHTwQh}Y&G@J`fAT=MMxR+i7RPlw|k~5U0S{`BkI8I&ZkMY{YazWLPblD99pOa z{*f@dOpouPCqGo?zfJ^0QK{mrlC5wA@{Af+>ZK1|VEB7dc*B5KFz!C7Qs^OLd1eKZ-dZOE|&yS>6 zseS1l#qYfuzB>HmvUQ6lI*!6Z^le5m=8JzLfGbj_gG>9Dl?u34S@;-V6^eewMOQf0 zF*OOl?5RO~OgxZZlx?xt!GFW!(2AvtS=avfPEG#6)RJkFPjDSKM}5;7&w;fn-8k5| zN$ILin8nVkqQv`8S;8KcCYo6+i>(URX2Hq++-|rO3p`{g1#7Vyd~0L@#PmF zq@9e{`x3)>@kpsup3ChQ zy|5#>nZpinNh!|RWL4VnUDQc(C|-5#>A@DQd9HmNEywb`^&i*&7%$NOIo~y<=T^sB z%e?xEII(*n@7qG@rATibN!KDotZ2!@<&N4MjexQZyR3koN{{?p%u$Lq(k6ca%7no( zef+4={q5RFj~Cu1ZtOF$TK}_9{q6QRUU?o*C_^oSmnoLH34P3R{x~^tW^J&}BtWo- zzr3)t-$PVhTtLOWK(&-3Jz8Kid@ACLcOBch-6NW$R}t2~`YYM3DRCxN6sX3tG0ac7}< zzZDB-g~zPLj764&y!S#Ic00+4S}5-3z@6{PJl{DxmOcKil#mY%J~&HXW2X^h!N-pRwGlm-6JZjGP40aR8}X7t+?BJZ%~HO zkrN%z!4q#gbVKq;)j^S54wuos?DbX^UF>8O-z7rSv+r~T(wLLvFx_ra;Za53dH!d{ zMdb(RA3gBm$QA8kSul?ZT5?YcTg)o%kerV0>7N#3E4p_1#k-9I!TWa-Z|ZQz%1b!* zMMt~rj0*}r%#B?Y*hDz*um^V;VI4J_*GAO{W$i_^1G?`@NDKa`DKQ}&qp2a;4#$fy z8N?S5++)V)O=>wxgMKL-cNBUShrOvebI#_Bnr`Bah`PWiVWM60B&$_GJ>Q<9q+7L7 znl*iBX>HADK^qIXxYG>h9}){#W9P2%YNADZL2+vo#KGJp#SV^#vRI5baO|?4`*1DG z`I*S2aN;uoW@|GAMAPK)IT}abhK079aas4~ds|FlB=#x$vs$A2oaXAHVGwrbD@th_ zR>SE();8asv4XNR(5`7bo??&!wbR!0EsV0tzBI60B~8;Vu7sFd!fQ_llDdMhw-m&p zC?c&!CA3zqd(PJ4ufNj6d&PUS?!`&DVAn1lYWCi2ZPwPBDqHe@i#jx>3YXam^MKzh zfffmrX*s^T$i~|jJF$0#Kbj8B6@`zAI72Wa;K(^mqHl`T_hp97t8`i`8@dF{(^hY| zhMVxvg!;s*<>6^2~@!$mO+7t0$f(Z4j&9bOF`2x4xN%dW1@YMB>aeURnb zNww_X+Pq_-6lUPw7ZbZ&R^@A-*KMI)b29f`wd`Pw&ik#54u1oauiHTnSHxHo(xi`k zF+V4zZhutLJ{=5;JKvn|iOZzKIJ?oRCwEL$6YI%IM{kkoji1Pji#4fSwO6RBIFha0 zcFOdivmg(ngO<%_j#uY!EbY@e zT1$U9PFZwl4qbqSoFJz?do{?6cf8ZL{N=5v&_G8XdKR9_drjxv{tHt&lBJI|vv67x%eDFLKV6*h-}qf``8&$x(C}Je%(M0k7q4Gol9h5{qn- z!0PVo3BuFt!lQ;-LHa;cg2xqrT{(fPs?8CK4R^=v@Khsup^ISZIake#%!HN|g{eUn zVjA8izCoEuXl-KGw2!;3*L^O)A%9wliT?B+ezDOvZOy*PP&_isY(OE^B5o!&+xWv$ zG<;^XBwEW%_Fi>8}mpLx53Ay$u1!2BlME^~sO% zioVo`)(C5+-AmrD8B!1AU)kV}Jb05~-2z5v;c`&&YHejqZLpA_spo>|ztxHAVxmk} zXtjotX~0>G?&O!7=N0bBK`X)vKX5Ym98Y}{?Tw584aS$r&#XN%=S`y>CYZ_&DlV&O zYRUwDlDwjEX5Xjt@_x5n2T3cuoz;4{)Ki&_6tPs(94>vBSyNlHemHtn&NpjKaHVB6 z(OF)Qqeg{Yw%toZs6FHLOohLkJapVZbG^jIM$FT<%7M)$N74Q}owcQi{mE+-2fPFS zeNHpm0+G74Lr~im$%!QTCC{{R$(z;A_;Z{(hqS^Dinv9(IxjXGlNqzeDSN>jfh=+* zaw&>g`t63Y(49(JKR2{upS&3VF@w{e8K&`aL?oYug`Xig!8_l|OGh!!)Eu^<&D1%h z{ow8|nsG3*hqJ!->5p85&y_>yaZ$%+DKrjBQubW^K`n9Unt2rY zE>)Hm1mQtWj)Eh|N0F?GX`KMuoSpleiKWOfQ|^al0^tv*2mPv%n>pL)U?o4IsMr@G zr=MG8uaTn;&xp8-rrgB9!x}Wa$`h*J?kA7EWpXl3aOyOlSBuG@R{AnVYt*_0gALo5 z1ab%@6Zu09fBRmE-@X@Z`>q;6PJpmL=UgpTVbh*}73=SGjqLN74LQ(YM;k`2$v_Cr zI|wS1l^m#LFNIMetUA;kK!Nioasmi$hfV^l{Zrs=X$ca8`Az1F>1D-Dh|bR%PZ-^iL&LyMH@}{DnIRF> z;pTGL?C_03XS2DH+W1N)?soD%#g@UksgJI+>C4&h1W)tj0FuA6b>L6(ejN1glA!@6 zA|wH!h>wJ>F2@79O$*W62jeh6Btl*iC1^oxwVO1JF^QvOgUOW(6yz5o>L2wX?Khj; z`nyfe8U6qQoGkn}J1Uc?{~{%jb{csE`IP|*S^xJ8__?2w3TXLLQel5cDr~6n#={XM z8~lTz@v@G>Hoo#*nSG=Sz1M({wzl$0(Mo@aYj)If5ke@9fOW~cmpH1|Eq;=8822zz zjVJffg=w2>P$RZ3S__6lX7w06J`omDt>)@_rVedm?YlQ5!>}#Z=9CnjFV2uZWTUb`h?^UHk>r1v6JuF!CCE~?VWGnj6#fz;+q}uYs zXdG4A%w<1>Jks^oUF7hS(#c}3LV2!u|4S2=W?X&d3p>91q@RypZhEmV*?E8b<*gw8 zj6xPO{fJyUVzZFENNR~8x6}N-j&4^Kas<$orgaC zDA(a*uk6X>!($LilkKSXB_6%d&~+fZBj^AcNYWI`SQ22*s z=k!De18y(9Hi5`d6PbO2A#=cT@0ccb?cVm{I%j5G(~s8I*+1X|@a!dMi`i^QP+yQ_nrB`Pn#u)*$bbA~aRG@tOkAZNk%W*%VAI~v#aSj0t9OP{4s*ajs;Jqa znwij}$f&Xc*VZtq3>WxXp?T)o(6ujzrXOTOS*-jC2T5BvlaefQ*saK&)h3IF8s2Wl zZdyN;IC`k=h}ZT<1^2o!QC%Dc6b$e88r6UP?d_modD(%xak|Wwalt1JkG>{%$~T*n zJ&CRL0gzJr8N`VI;&n+bMy#AN@*8RcJ+Fik#huS-oFkMT4HC`O*8FVj$`ftq`s&@d z+=%Rb*QZABzS+TtD6{O5pNY4ASvTKKk-dKItYKxLgQDgle7rOzKMLAU6JMA-?ziu5 zWidQ`E`samYv-HQBO5{7>g+FqzEOwT&U?$DuK@g}WyPu4pk;~&@dOLcz{<h)`r2II`b+Q(KS=FrRrKl0_j^HoH+I)l#M_1O>S*MxvNw!FR zP27|?LF~WZgEO zD05t2{3VI#@cF?+#6)O}fu>CVrDgjZ0Y4ShCk+El10CeR3u%UIY>rC3;aRQP+@Yme z8>9KTL;y_j!mRhPd4gLbU#OBY5J8j76PQ*@fY)qc!v|dBsfnBArl26|u z1ZuyebS+C9gVBvVf4^Arjh|-L0(~L~^?{yAB4n;>_LYrf*_3SRR}D_YAP|{Tx1h{6B^k+foX-@7Mj}^q96rF({g%^^g7=y4V=q#e8nt=sJ8Lk1j=+)Z;PSxT>M34kx&P@ z#nkd#=3pqRJM>wildSp7Ti(}|58{aX9c;__qxmB)+w#}ab8cfiP|I-6_2y>|N1fU;ju*^J$D^$>RkhusI;$}>ZkE5$`x4^>+5hpc9Ga6P z7)BXZeg0M9{3ds{8qsPInV`v<{{^*tQ~T~<{rsi)4;=`X+^?|Fvj*|Up>)aNg!^(! zEx7?rTRY8zkG2*n!+TYQhL*&Jj~%^c_>EpbCxE{y!hHg(btB8>y zc5b}eX$tU zk_s0NIQE}O+iMEHlR+Fpq9pE^%IpfUsOMgkBc7exWH4O z_VTh5laAEmj{CN$EW|j==c%&1ai*EqkF34QY7?j`b7oKI4#YD=u^H<%;^sd;7(OS8 zQ7w6sY^drt?%jHQns&y|jnLY#IOHtjuNJrCzHZVpGNF|dftonSRdiN4WS`7h#U<6( zGvtiNpXYQ+%$zyYKATun32az)S#!U>bKl*HpI!gT(cF~yi6(N))H9+^Di~novj3RI ze$yy&+LZOJ%baF2y(OZ;>$wux?{sDkXXks#{u&vc06xY`@cB_MkOV!vJ4ur@tDIcc zoU$|+kM|5-kJ@2hMUuw@1{)~MxB!jd=lloa?%u|&U_3>=-{;vTc4xW+WICLixB&K+ z$2kLZSbASEIesE_KCjn8S^q78AD_`0;vq!9guK;;Hk0@jO@m9W)j=^&H>USqk5p%N zdOSN>Y*%L=0BQ9{J6sBpeY#q>Q=1&NHt*yOe;q_w6OK3jra30pDW#-ObOUsU$XKdq z%~5XjICvg)gG-x>Eg4>|3w+d|v@AF6Ny;B>ffPd;VIdP-&Qcap&88H`Jj^o{KM3 z@O+#PGGAIE9#wl1Jh7P}rl#FX)n9Ny|!46rRY_zl1r^id@AtT3I^EwSpusd@eYq-*hKK zBi87_1GmJ`nb7!}l^XGyf#}g$WMN@-CxoCpFjnL(RdvGFZ43 zvvIZwIhBQcx%hW2&)U61<|#jM3;q@D8FQst4VEQ2m}YNIjSne?`)lY{m(>0UfM z-~9QUREL-!%5doaDyTf!7_}`3Vj(;0%v3>0yZjaBuSn~xX#(kgwfEgoO=j!*I2IICMtaXEO^JYjR52NF5CaDpKtO6lL`n<@(p!|? zG#V+=l`1tLy#}e$n}F2Nk)BWkg!tQfesj*8IcL_LyXLNS?;rDrxc0YTf8Y0hdq4ZF z&l^d;(qQz!)(Zr(7!i+B5&23{IqRYtN_W6BoRt?ypYG}~ke#k2~^)NSAO_55bxWgawEi*C%;(&7rogm6&K=e^RPt*+BlIns8f z)uc_aeC`A1?6xEpBNJ7+7|Tdwr=v)^cg}haE%JFnt4bUs!UU!+LC5}(bU=H00Auh~sL0fq5zQM0zFdsY=shXgC752xm2GNndy%65=!Th+ppY$%#CV-%}LVl4@+}+0Cmo%X#JEkCQ&;WsgrBnJXz` z?e>AXK_+;ila-o08UspJ7_@i#h=#9p1UKy>mtD#L=wP>gp;#x^`S_UJc8lNtaL>iP zh;yUKtEEN~v!%wU_SiwPRG_B5reSJAZsnSHvkRN4*QvOIzkL8m#731kKt&M9IP8wC z!cKQlvdPO5U1x8+Ra6HvG8%_zLxSNifHPL)l zwst{Pi%jtZAqaMk^1}HtKYzFLQx}voQS`Q9%fp_--6gdSlmQ9#9j36ol^tJDBm(i@ zGYfEbuy{iW9RMtLx}G-N@D=E6`KK01-HqKJMFSKLKa*#prCSu}rnN929%&9CEmQ0V zRg+gS=kAb5*>Z(g~5 z7<6#%PlTDpdqW3khiup}-($n7bf<+sR_R%R-*eO&O__cWufab2fJ^6@Ah`K<&~UTxG?LviJ29nXnXI zXK71ylSm6sLqxWr!h^`M?NU!uzB6s6Hxwg?*LyumQYx!+Gg`NmF5!;yl;g84dtjVI zsu3k~vIWMwu)T+@A3;^CcY30h{HrQ)z23d^`q~j zIQ9O?vNE*#I7+jf`))7i5O4Sc23fywoTEH}+3RW#+DnrpE?@L1C8*KJECH5Y{s4&( zej}*E(&_lK=<~8^wjMR54B3?P@+J7(#Ao7&&wE3=F&|qNRpq!cg;I(Kvb4jD^5WZC z5)pin4Fm7m^V_@Ye#HIgpX$C)HhuVN;Xv|SP0q*Sr;th4dcf*3d^O0g3hJC;4HJG> z;>%OEYHIZ~FYmjHal*R$uL);O%UZ^2K;qn_+>;uJ7o37|$i~<6#8erWvzO+;L!41d zsD{#*p`TnFYU92?7jEyk`%w9?pNU4u-PiMq_D+#GBUP;X+16j^a&JaI?_898i@!g% ziYwanG+mnUs&N!e%zZfQJn+(+1znNfpK~uzc|aER>iP6yUhSS==psz$^PW}^?d+01 z3{u0L7=ZG;=;7V1urYDcMQ+ee$}zy77kwRQ3u4cVi@$h|F%{N1siigk{%$d=vz&(ZJHEnk_k*#)kV3O!n=D>^E zWT0#Gc90mi&!jo83l{)Gma&PrEky(=!mVHEBx@cQO~RnMV9$7t*2Bf|3uMQy!bqJ2Kx_ z{3z)@Y0VGVjguG~!Vr0c5-Q*KQ|~37WwRe*V#FO^eW5cr)*^(|S?W6s6LbqT=}^R2 zf(A~m5$@g)sLghD*L}n9X6C|a1EueF6fA@f45TXdvGGZh*btnztt^RFBuE+#>W^O5 zvFl@<@w?Q#+DsqSeC^hQDw)hhMK)Tdy}7$X`hw-r*^dtkui2WcC<{CZ7{%(| zO?4U_>~~~NDH^Wajm+@>Fp8_4u``)}=C7|B^nRaD2SN|*8(0-Ny51PeK5P>kkTB}X z?r5{46Df$W7hr7@R(^3}DLi&O%GszV;l}VO;W#b*!Rv9_%P*_<Fg{B7@;q#7Pzw?=lct~i zH0v_Xpy%@V&Y)!vr+#g>o@kF>C!>&rTYw;J)Ixi*_L{r_|h_w0_fESjn7<1WM=7NeD=U_a}PR4qudXRvI zq@cPF0KN#d^-kbg5fBAllju1w5m34~2O^_q zCX)7<;Ix-5st}Ty{q9-*6M#fy13a2iguyUBV$*8*=odO+V^SNfRu&jHr#z3kfoSn> zuhIR|5d4q)&-Q`p1MbW*ZNE>4#4IR94*nbQ2){XPNtY@`eG0(*j0e7*a_u*)BAS-K zy82H+i(g7~+TagZl6>(`J4y}uq#Q<8UKoTPqRDJYV7-_u$O3h8!{Vue`}}VpQFGEE z=N|aJJGk|mx|A$9RG z!5=7LEX$&vRytx$H>GsK1@28HW;R=)`zw;#Yh>J96Xq3iFV+-q~W&px-5 zpcyU^L^vaZ4}gi440^%eBGMP5JbnL`mBWxS!(%%oK?WHM_+LpT(Be^U-VfTJ%xjc_7H4UB0eZ!(5*AD0&u zvUfOeGaIxsuH(voKr_%y^L2@EFyoV7yj(EN_}D#dkUr4vnI(^^Yzk*b=}dJm(qG9UICsAt zT2)aJ-BFnsvJb^a?#odmTy6p5;Z5^Y{|u6!o_bF$Sdlt>KlVCiWv=E$_hl%O@pDNm z+8=6fVZmTr&%F7R_2|#=Lnic1#36Ex$YM!FQ%t7#%RHkxbi%BEo4x00jUz9VkFW|x zS}O1a|1h}QKcN<;Ai_jRL538%EKC;g51DnWdm=8V7Uiv^CE!s+i#^wdo2S~PVP^vM zeAV@A_T^B+)T|EZ%7&7hb)zes2_%nR^_<-CxAVHPZOYid&sQF|0v|FO^Bp%h!?-)l=WL-~6w(AC_XN1@dFA{d52IV*avSW2oRFIT1;|hOuo}$ro z%d$g*QSiWEH);iLLtwFioJk?e#xVtLW(O~0-gPEfrKz4ykV#VTz2UImsGuOO;j|lg z>?}EGg$?gk(WO$L0cZMLArWMMJ;gG0f!nR9kmCoxdBuB$1x-@MPo7KSY$0Et1=xd+ z-V-T|Ah)tn2V^LHLkFL)=%4}6qI|Gdqwc1q#0s~odfON%wiTE|OE8v=Yd_p8$`#wf zMkg6d)ZU?x^RS$5T_(3T*Ged>gKrGrAJ8f;CUtC3Y?AiR z0h?7_BdkNrWc6OC%0t%pq*u&x53e7Sf=@+acdp45+*+)t+>x+>7p(;s;m=U&e~#*^-;01L zJ=e;y+1Wnkd136Hc=%gm$B6rAf*eoaqbEUb@a@<^(5)HD$2^T7B?WJ;2;Yy>G_^+< zH}x^{wVy?|szlAB*dgj&0JU!Y3$9)O&|s%V4Wt5($2XuwOQd4`ahgi4i6#5epL1%u z&{r>iYK3u-iyjPcn|d_o<=Phme(339o&_1x>r;z*0hSe!a}KO1UfcGD=Wkz&K63+k zS&!**lD^MFqpuq4_SNouq5GR}HK$K&qTDwG$NU|^bGzt#TIB}JpambPS~<6o*V5X> z@8tKbB+6UH_)NEUYJ49w$8bsR5DwKc)sWR6sN~IXMMjj5M8n&R@5sRz?`YVb^C3-^O0~e@;&A! z4C}koC#JkKea>bsFBqhK$hiOUUIv#f&aShwYYm=UWj%Sa8Qyw9r_G{6@zGrIic_DF z*>Lts`Ha5>pw>)eLk4n!(ivtK z#jd`;NNC5kZHJTZLD*Kk(9Z6sGp86UZE^*1>!*5h9P7FxZ>c;{N#T2a=cLoQGL56$ z-lMwLv*%P=m77gd&&0b-_$0{ZjrxF_g6{>m8UFsgQokK}&A&R7S!plLSnDb72{OsZ zlLKLvq|HiR>V;{=q=vedCGWoe;WdolZO~JiZ%TqiYkElBqPM?Bpy*V#`$M^wnHz#y zBJ;aGccd#_)YLOGyu8T5F(m0*6sN)wjmZT41$%yASE1b5BEQhN?AB#VeW!fHO2iid z!pHTZKm-1#n(_@de~nf87_w9%i{m=RMz^@nYr>Ag|*EQZ{hW(HBI9%Xq(#^9T}(RQAdvImUAHllujSBZ_vVe z3^v#^7*f1*iuma32c!@3u1frr&+dYED7BBD7K+)XzK*Kpqgzs=N@aXK1QmWc1mAuW zT^V5BBK|{v!2#kf@Yqu1fEMUR&?5EpGStatf#T)4upYt{kdve=#v|*fqe8y?w#M?2SWa zUCZgA@V+~+8&_P&wG5JpRdo>h&;oTcYgoYWZvdLj>PuF>SwHTn1E)$NK$&N$;Yc-t)%e z-PDEb7@I8}+iNDI@6cqHh$m{;4MD@6P#c#*P4~;V;*PP=+KbVTsH@9f6eA;g>zc0Mr^Yk}pfdWkvBQYp)m$6IXs zM{p%A2~s4RkNv!!a|y^f&rw>AS`U$QleqgsG;v!R1>Z9k^Xy(QV+8p)E5Kwp0Sp)m z*+S5N;oV^M=}20o8_k&_4NS{%lGZ^GINA_1 z$ykbn9Q8RCc>O-PgG{GHb?6F{;GXpVr%$r@zs#sqOp~4>gh}XQjQS`BUyCB<&x0Ry z|C9gz7up*Aqlu9LmR5^gwEPv&uq$O2 zz))jZO-lX&!V!;6jEKR#p< z*g+YYf#{KKO&Ri2^dxiAu5iuCW_wCxBaL;9au?$dlHa`mlKYks>Igwu-pr2_I@wBmTQE!kMm6|v#$7Za4xk>s z2>><1$T>JA^fqRfEKvqC_+&MT^I)66DZIy>iPe0A>?{HKrM4E zpr=Id22nw>Bo&1Iu`j&94A@JUVx%dio`rf2Fms{+gs_44`L}UH@*HOK5&-GPZPa?& zxhM$DOJhy#L(NxtVC4=L_m*g!Ns}B0EXc3cNuC5C&iw6Ii07ZL0wJ^J1J3Fxqz6oO zgMR;(J9wWzGR3c?@h?-j(M10n;!yn^alEJX^MCume@?2zowwkf^tJ}6lJGi-J>eQ) zh`%>M0_MZtj@8A1Tjm-AptT^h!>29)xSDVHwnCV;-)?|+`XeIwJ*j>b33wrIpyi*` zB8AaH@Zi8V;Gk&#`P9z z!>9Pq2ot+q(KB~|TEhQklY&Byf7!HHN`y%lEf55$NnDs$m zLn)>zmIS0-vvGN`@aRV7Ip5%=^fnPMVD zmL`U=Ym8->;d|!ollpx6{P(@C-*x@2>-$HS`#tBp&+9&~`#$GB?{nWpUoy~QXF1Bk zz`($+qkX}cfnnzx28JDDyBWbRA%{P{1%GyU8*80sDC^*z1P^~WpSyUDfuTH}m1@rf zp7(faTX-`taG>e`cC>jsyUxHMS)_B}++{!8*`bRMMJFsIJk5mmXTHR;IA%De9J-cD z82Zz{`Gj7|2gi1e3wOEpoQQFIcvnP&sC)I&;gfa5Q`#3!#E58E{}H`tA9PGHZgiEO zUqW-;(1MoD$Vf&+}w0UU)6NUHye>*SI5b~pkJuA zli>{;iwpz9``rH%|9|BHm)eIMmS5lfRsP@&hB{KIpk4Qqusq}39YozzI0YHc%V1(Q z?M#+YyW+IMcIAo`s7>SEBnsge;05CJcDe)D>|s*Sh}y-WNAK`Nd4h<0vqKp45V$HfB)4}&okvhQd8uY;h-~==9`yllD z1Cln7gCN@wuIoR2@)c+a-ro{-st_3C#acUPKsCB+m3Nn|VR}YTt1W>1;F1>-DL6@g= z4KDd}0?i;GVtpG>fTsxh*(d(sv7@i_p;0FJBot)a(x8caP`(Mj@%b-kn7F_hBJSn6 zE$UbP`1r)$6(VF{*5#=>x)6fB{3Tfk8aX2Xm>_lMn$0twBAuzFJuRL}2^ey&h3bdjkH@4{M6Sxo+*#XHZ zeGx!LK(b690m$x<{Ha$1D5L_(@ZJI-b3pRJUM9_`$m%WnORzxE60o%Zf=s!1jvejY za;hTnvkEdVQXu7YY=8?~;fIuqtfK-Z6d>i6;D-PcHjr|4la_#q2U`H$hX7bL#Ket) zY2X7TV8a3cI_6x*j;3wRKE4e2jA#}Fz-{9K0JbdlR~_)NmscQwGw>b&kO2WG5C`(q z(TtE29K>+|ur3I!s#68XPa*kor#q0nAer2O0P=ZAPU+|X@_tB`>DT~r0W|w(nKpn4 zYyn+4+?l{w4)*jd$GzbLOeP)!DTkd=RFJV|foA`pKNx;PUKx@@Y9oj^&ufsP1TX%N zPw**_qK^qa0H7TL`-Hg&nCQNgYc9d7ta0Hw7WDe#AhCxv%3$BVe!!}{EK|Tz%Wabj z7l7HoE&%$;Xk55p3|jc;E&D*rNnmw;BYKlKXaYVXzl+{fZhYZFK3FfzMQTV}`AZM( z-@ga+xxzx8ipa~qfB&7Nf(&m0V`Bh*ukeWz71H480sU!J=){RGU=rph^w;FUYYzc* zj0t^wZZLi;V3|o6SHi^+bK=B_K=1_ihZ3&1Xz*kLCRPiz-3LzxKpO?L`OdD1^=IZ- z`>z!A4g2E>HK=v82%dWTo6<6wrxOJiQ_e`qX`s=B zkMzTwyxj*Ik~`D|Rrm`{t}3t>dH3`uUrl_!OROk|J$#2=p<1PloX3#=_}iKmo$0TX zVQ=$nM)Ni|oY&v|l1E+!v5?DyDBndwsXbhq3nCIh)J4PmAY%6jwP5o*((7B$R};I= zkFPT`#&H4O8TCf1o8Ow^#kZzTsA(WZ7fg_4??y7(# ze2Pz8{(+kcrHEVPU?&F~DV{D&q7(Y+3uSb3ZXgSJM-VN-6}Cw|VDFn`I)=`nEvLg! zUd!i)QK8ezLxMTM(pg`e1hRFJsP#oTiN~=^t!~;PQQ7R_A1}6C#mO4ML+3l%M zcp;8jT4ZaUdocFpcn`US+b9kIEj*C=p7a~4$ zwxWwpY6^_G)_Y=jRP_-J`@Ctfv2Ply?Ua`ZEK2yh(q1y)qVf^4ZkP&+nJ?tjjC93R z(3p38lh^Fbb)V7kY%P@PymRWYrLx0r zCt<6f`mmgr;v4e5e}_@2zT=s(vHt*0Oq?9b!@p-BoBY=28b_!l=*`m!qqqFU02J8J zrC(dVEKYIVWsV$TD|(XzLj@>HILOK#0thqNu@>Rqv8YXtcq(zWA!u2mv(a=;*-a|D zx*AwxYGx1ZDV5w(DBeU^E3tL4zBd=osEihDT7Z8R$RDNq<9p=#P~eKC>i)|cyZ`S0 zd>JrU;VWCn+1j>2F83X-Umr~6L9Kn|C$vhatOe8(x>(6}p`$&XfgNvI;f(}4O1MYE z@^WZ!cgQ>pAt4thi1d0qp!~fYa}~$V6!4s&YHb^>HaX4-*r@-Okc-(BxRrmuf|TK~5I(WtVw& z#c+6z>c&(Ir6+XugD*N-lJM5NW5cD#%}o;6hdU#7GF+-atwx+&7(1b*(_;E1$aHv&559M=H8=oa2iqw#uwMr?=p&a%{!g>H$6# zGZ9BgVmPC6s@!RJNf>1?56o<$b=LEQzCUd$HVFAEoLptq0vU1iFvpPi8Iq@c^&HXO z6&*3nByODBSwpaID^2cw4!fpXTx!m37ed`*D1qA=+jK3o?WMH#3gv5f{FwRXc&Q@( z!XvNQwSLZ>8L{m2{er*pc3ktb5xA7`Xk<>|j2O-()eAdp{3u7K>3*iG`=vCP|Fr`( zgQswKR?&9XRMz(b02Q^ z&4|542Po}-{q<`8uB_hdE)#FB8j*SB-9s`pZ%C6b$D20w^S(dMtgKO+xX5I}0w21) z=$rSdMs40kU}*nM^044wz>93*@N>)uiN#bQn(QJqHPfG=1(<%kw9@-Y|7Ry)kFLkD zF`Syu?-o2nAl3SGgY~(42NR9`ni+%R#QN;89W|nTb{z{w%VoyKO?n5`zYjA_0jJst zoXTDWp4-0#|Yj`G$&!W4|b}dGpsgaqt7r&C*SxyLQ zR8rmaoK zLvrmqY;sLH$_dmltR-4DYb4WXU}0hN@N(Jyq&czllD3?M;Z5n{JhhbzVTe;RyDAce zbh;jZy0lN@k&T~e8aM)>dkk==dQDE4nSmQ`heW_^ePi$}M844|6-I zn^Z-gdJGpaYw)iaT)gukBK7v`OO`b~#>i>XV~K93rPio3H3oivO?>u}Y-qBwypx1_ z)6jgWqFc;ybhPBt%3o+LH)VVZ-l6*GZ~AkX=5ju|tRJ-`E{BTtP)&%@3Y&{&R5tz0 zex2VaDWATJiB_r=wUv&HB~&a{RPjo%zzu<|TipNTiB39MNM&#T0+H_rV+e-2abM)jEZ<3nDKy1o&))ouZc06=T0Kk~(^b80tS)N}6zhF~mCdvj^$I=lxo3tKgSUYRf9c+;>;kd3V5_x|jHYMUGZMm7E_@k{!dH-5n zx5Erz)SS&t&EU~_8hUYZ)cWskG{MljOA~WuG@dr;FyG+os~cwEn)<-n?v4*rfY}fJ zziJD&)jw6S=Hs6b!ciEq5mv#M=FaGe>xcv#@9M8XZ5mo>gxS7L_+)OF%)!!sjs@P- zWM$XS&Qy1YY!E~EmT4%2?l`ta9%jrAShkXzOJTy@yk6wx4UuN>COkLLvI{ldg0tc~qbb(%{8Crsht zYwA&4sU~NO#`L_c*^n0uO)qSS;78r0bW^`~B%co;254Vb`mx$PAI>Oq61=Sjc4DQ( zbHYf+&%ib?QsSYOupsQz(rw9+M=%-WX~NljCerch;gDM23H5tCiw>#$9)3jXod{$> z$Tf8Ln?&O{UwY816>zbUxS;&SpWi-{K;d+~)c9t#PO#&ebKZ5ldd13pb7%P@#{;?K z9W5(wH#grnL?lfZHvrH>e%7! zcp|S!(h*lSo90dExHUQK$#Ai)x>rZ&81HnwPnoSj`Q&KgMZy^RcK`WC_gcpZ^@pnN z2ElQiuyj*WvDB^ntnjE3m@rmUrzG%rt5yl$Cu%{DsYk<+h~uh`X=ix^SYb(2@|1xr z;=7b~fBhR_?{AFRNp`PKHVWlXyQC+>$+XUqG5pHOTScN&T%YZ!dMjI%kjK88)%6!_fYwcQ6!X41;#&iOv~?S+`F_ZikstRqvxv3$rphb(FVHZ zNlq(=iB)rE*9z(lDMe1aO7R*vf=FnUn2F%& zW?pJRj~&3Tk378bO+Ce`Fe_=XFIB|=Hn#Ga-J`fq+4%64q~6u@pBx%|FPU_aDEYR0 zA|on}L&=WqOaX_~jSb%pkFlS^F&hRPJ48&@_z^%1($+*xL#>Tj{#l?(m8ebJJnf~T zXT>v|X;?;BRovK1NPB74{?e(S+s9Iws}L`?RHubu)ayEGcfYE+Vr=+asmC5pscn=b zyZY!<^RP6IZSDm1g41-1o|5&u&I*erbQ+=;wdzE~RLj+h;ukZ#Ci=y5IHCkUrYx|v z!LE(9;{6rGl2vGJf+M1Kq=hsGXBPpNScwaM9f8be1XQCuHdklKBWuSCUj$Z&zEVRv zYLgb-dZza>St{KB5z%;+C`4F?z$9A=UtQLIQjyM3G?o$ng7OA!*huXSbjj>yh3hu` zC8LCo&B~g>YMbz#5X@3sEV3_eAmj(_p}2gi;Z@WV(JApe-nBF0rl~}&?pccOQKnY^DZz@LFZw-l^MlRb zqelC!4A%&Cnu#8DNl2Gd>4Dvhlyk{z<|*TZ#cPV8Z*?255s8kJss(ksz~T_I(rSg( z*?U~^KWzTpFr!lm-|+KXUW z@I7V4R5ch9$2e5SsBB$6c)CF^48gY@GjDdC6I8wyoXdUoR#lRD#vR!YZQUdZ_p^kY z35-}K=Dcp#WE6g|cGZC(ahBUXe`!caa3dSbE9~Up{=W4^^KK9PyBfCmR>G||G-Ebu zXQ}<}l|l~#iY>djsF=Us@t|*3@|NZ}xt~G$rll4Ip2e~ms+treni*1tVIEn#&M(X= zs&iKBSY*uDxA@e~w$0VCOti}&WoWnDt0o8L`2I&c^cePTbyqgLS zvWM21ygnbba101D(CVyQ3cbxrDf7I~)`lbh)5VBdsu}4t%nNxPP&v6=rQf7Hc;~Tn zaQijAE~BX9f(<9^C4HKNO50el@eeJyA7$IEcJQc+xdyj!@Kq*e#M;MQ&h@q&DWg7T zN7bz<+BF>_l&!t}U?pDjNhDIF4FvcIRZ4-grFu`@(Z>ECuHJ0at$>+GKBsbR+`;jz z62Wj9-Mw(FbJ(;?s$8XnJ=t71YKH0}zb{g3Mie*sr!nkG)E6CB+R7#kO+NSf+iB|4 zxa_xS`FR}b8&xWDMMXwtuuvn*_?`hvZA6019=ws4Pl#L#{}!R|SRVm3Ey3puJil?( z?nG7+^?cS>MbyH-X~StKd#B6t4)ZZYNo0R5rX#9DSn#Eno_z8}VVui4-hy4uDoOT8 z({N%$d?EymUL}o6U_6I};}OZIXBA@$)x$rHc%;>B>_q`ET)HR~`A#ay7I{DkL)!jo z(7HG+lw)`tUuBfkcd9S;F9g@Lsh1alhD1`hHiIQSJpJ~*(DSClCC>u6-+u2(A>+~SfI)d>q#i%!!Q*-<h;EDC}RlHEtn!w4u4y1Hao&uS6}rA;Mw8Mi2{JRygX3`{h78 z@JSoNwr*h%z-TXP+=8xd;<-~O>YeZ?_F!eX^sirfyk`{SNt92<9VgJq)~ApzrCv!` zYXBVo;h(2Y-FO3P1XSGQ0O zwY2SRlmHh3-%ESs=GUI5y_Wc*zORf_{t&%!@D8u9|bi0RhSVys3ra^hBYJQg`P1{0Shs8jW zZK<(+VV{<5%J(*1ANSHF%=jJeX-f?qT`3;tm5f;I5jG3ApMft)PeP~SGyM`kgGqt5 z*}kCg?nRJO%e)_=;{D<6U4A=0faQs1+E87?#J4bZFP5uQ1zLm&@|#x8d&+1M+aDF% zHwy%9CWSs+U1{Mis_Nzl)-kNMxPE>or>ZGRIXK`iTN|H!3fpJ7pHZwBc*@Yz5cQzB zW*7V#JE1lxU9PEUY$b1^q-Du;mbAB+gc!Qq=%7+0nsYdW=ql0E`MlaJAhL9&!b9-K z5#Xl|N0=Ze0rr3)u( ze!m{@N?^=JlG5B2jMchr&^4#~xE@W%>$U8fW>jvlwi&)F)gYqW*=yriO8)+Clfmu3 z=Y>5GSJQg$gbPq}ZlCe3b>J+C{vw7EL)T;S9==FbG0QdiX`X*w zWniRO$SI7E`<{{*iVoVBzm;=M+!QWp~v$pr}ZQnJ!oJ&ag@n!b&a_iS^W}Cua52M?s z!~gAtQfeG4`EBUzP>P*P-!!t(aNLJ$tqEPM%r;hRJeRozW?y}~n7vu1dok$Vm)ExJK}@-_arE|(!}Lc#V`8p7vw$=qORbs>DM z=f7+a^keR;rMCl(Om(7rG43S~d%02<6%~@|4)^Wh0hc>|uTwrfT$F|lry{OTmsOap z$p^Xom*cW^gIc;mCfoXLouSj+Z4!-3w*H;!lf{#x!c2Af4Misg5wgacf7sjPcDLbY z4-?YxHU$Fl&A4m9HTVw_ zzJnWuDKE=usdO4VByfQjyHqDW3-pPLE6T!Mk9nTS~)R31K z@?tP}dd}9a#DLTCl~RS7pYDHQgcqqc(NkGx7wMiY-EiN^MMn@X9=AO`5UY zaf5cNk}saNa(m3RiXBKOqg%BetM#<4_BvFW@?iMB^$O5B$)&%K=>f zCxS$%)mAE}^Xo*IY$|8a)U=#?4$ZizE!0yWYqz&M&22|zc`-c}&w~v2twKI}-lub+ zc5@wOpPmr6II_`~Ot}N)uUJI=XFpW>$~=y}NWY*DbOw4$gK0*B^rJ|hd%<%x84=qdZs0zS67Cu*JEqsv3|94^SsZjhF^$_VCELGxjDK~w^loxh@l z%kQaKw-DN;70N$chz{fu$c6F+uElziSScMVwkZGUna;(>v2%2SCPAuOHudxoxMr9N zXXu6Jqx3YlZSaau$dgdOx##$o3}@RQ5s92fv2pbNwep{-;NEtbD~cId-BShB6fw7I1Rt zq4(bbOixacJObAbQ+a4!Hnawa=1@!$Re}W*%aO{9-IeJJv=s@&sNF=AKdE013)9tp z-+GkJpu_vfX5ydrl0*b!u=eH#tgqXs^KCd`Vl*f8n@1C6svo=D1Yg#jcGcnV*%(eU2|?7xM0`=Dk;40kP`moT`C&a| z<05gTXv3@`nr`>a?PE=d3R0O+~v2vrW<0_8h}CruYC5A2alrq}x11+_jEas$%V_vlZLz|(2I zS#ELdFxy7YD+l-Oyzswd@KrbKC1+RVjh`@iP*EBKf?%S8Rf0jG$9DV#3VElH_qz#oj>4Bz$ zph=6~^bcqPEzh8rF47C3s+@loLcQXhJBYCt+G5-P(g8Z-dCxS`T3%i2Vs zk1kYt^}&l3jP(yxdWCUZ0Wukq*BnQHYz&oQU49=2X5(lC6&z+XY6eE?E7bujpRnhqI3{WTnDo7K`D~VZV!$O6L zIIX@zv)pFqpjyGCGNA$K7ZDJEh>acq*arbzt9t+d#36toI0XRQfdHl^TL8e(Er3q~ zT7i+NTcwo6F|o^R(_0m9Ch6P&V5@%exws?%*qXhSjb)WH@h${rZ*v*Q;gIZX155xm zZ+e;MEgNkhM?K~%IZi+Im;PGjlf7y zUW0^AoOvplD@WD|4`~hO(S*_Ls0031!;^6W70Du8R z^$efVfbC7)sremw$~!ZlIZi7BPI960MxiEVp#CSQOWpii&}GYMXpL8^ zoUd*b6Yj7(IUxNq8G3b(iZ6Io0|H{skr+;Ny{6Gn;OZhFU zqz!_3XKj_88Vv(tU+;$clqrF=V4VP|X~O}A2Q`880^Rp_s+_<62|?+4gUNz2Lb?NZ zbbu~9x2Ws@D+E*?g2H6*n*;qd0Kp4EfOkLB-+ijQvyIL*aA~mAL47jNxg5-eK@#{D p1A{bJr~fDZFL~&vt#9sGZ&Q{*)oo^g|Hr_fqhWBN?EKZg{|BW&tDFD; diff --git a/arrow-data-source/docs/image/decision_support_bench1_result_by_query.png b/arrow-data-source/docs/image/decision_support_bench1_result_by_query.png deleted file mode 100644 index af1c67e8d6751ee01859a0ea505d199b7fcc9fbe..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 32336 zcmeFa2|SeR`#)Z%b5co)N|Dq_q(qW^nRbnBicUxfp=@K{&D2pe31vjKnNvcFY-MM{ zNKz!i*d{YH#x{mA!>s>j(B3(p&iAwZUjNsp*Nd2Wp8L7)>waJF>wR6&%fs;J?M*CVB@KJ!;xS27dY3>44FJMT_!d z1lhK~0Ke0Z8=gD@{6GKC+}s@ap{uKFk*=48g~gfV){c%2OG`_O#C9AGHyW~Mk+$dA z-_gKZz+tD8-U9EAhVD6i`t%}gFU#b%&d$!UxnEBw@=qKI*~1@D%hNKs9T;;gbT9wN zx$*fUX!9=t{0e*r24AZm4Ez8b7W1ToKLKqI;MbK${DHT!syl(7?-$iv`=byz9+-ni zR3`As()~UbA;||^Zq5&~%ks*4!;tv{#33+cTZEx^JaCw0^4r|k;f^MwAo+oyxa|px7_2Hh4-RGVr%$c zi(kl0ffg+aL4pq-I2m9))+-E`wkYWHJ&=4$=iL!i)btbA_jZDg2VRTins?DpKYL7( zJhXBB1PiWwSex32S(l}2;&jUKEtXN(ZRM%CIW_xb_VFjDTSFOWc4m5<3Ejt^0b(1; zrN`-T9%`}Rzz-buD}1MP)volo0(24<_+I?q{$4y(ubA2Dv>E>htr`@KhJdK&ozN%@ z${K!eM{$k}RftI)*nHo;y_I}LZzM=V2Y8!4OMBenEw5*aoVlj$T!XSk>E`FXYa};j zmg`;HHcK91Pt_jM-fmy#wE4wIZ?CX6GX-u^It_M3AD5zxL%E{PHEII zGP($Quc_*q#36>ONOLB6XlU#XZ@^}{s-xzlCckq0zS&tPzZ6CpJou0MwoF%9}F(|JY39%tYRyZ?Tq8k}(u#J!!x|}pMWTV+ZoybI5 zMZj|oT63sGC!JgBBHgUkDKgF5b-<=9U&zPUdiFj8#EJ-QDJ9juteQBnDic^JS%xZG zxr(ycZF6O(i^AMmk(goBQRGbhxWfj@A8xM%gL_+Da{Ghfueu^$M^+@;Qx~I+b?dFJ zOH_Fuf{V|D2|W@+`xyR8_G8)ALf?Y5BRQ)jLOGjoSKR6wQ-eYF^n2{tP(s7((-d7# z8BgXWTT=MT!h~LpnD&U)NF#m8a#&G@n9e1xvYb{(0g^XZSClV2`AnMICWUJHsiO`R zTzqrPh4K4T_g&s`&7$#rZPL0Wz&70r-rn6zPkNkBpj>dfJJg#HGf-b1ERogwu3#y= zm=;!fV|vAt4q`$?xs*JS$jF3nn}$coQx2n5gql*7baUj=8D2w6xfCM2|}wluh;lg^-)$j8nQs zMRaD!^xZi$8;bvohU3FjSY;v#oihGV6x2=XV3k9-h>jR61IDB^sM4aU*dC0tm_J;g ztbp3uk!Jd(@v-4(P=1^}-+u_VLZG>Za734R%W!A~Vboy*Mq}J5@zsq0(D|tvsZOkY zM(5f$k^B0hFG$Zo^2mcAoV&OkTdAAW`cStk&FEqM_%=*5X;Z+%E>%RiT9^3Ey~9-S zHe}4XH7cE!L@KkUNIs(Fc!659CJmzcLQ|VM6f~vLRoX&|hU5~3p;E0eFIOc)K_jrY z;dgm8t|Hx6BAu9<(2K)mH{Zr`&}Ll`jOP3F6Ewtc668tT#+K>F_+3%L(=G$fA~sim zUBp_49k3s9VRXM&49cpcK{u6K(4Pd?bs2O<&?Z=;Vz5D{qv1l9oK5l+&@{fOD?P$& zmgJs`cJgjk66}_t_J--)f^h4MqkVhqh5K!K#zV%VUCL@}ijd4%T300|G$~|qxi)o! z)$ZmK6f@!_?49ligq1o)24<^@fL~MmiIEA1dELg+XD>CIP+y|UZbR?cj+V!s~c2GUv9*}luQM)>g>)N#hb&}1uD#gN1fS!upBt!MMcT);wi#d@n z`e=j$X{Otb=ulPojK(x8C`_+<=(>tw8>)VcZc;9-OpOo@ zQXxbxuYJ0~Q1|)eN-ELpsLrPN%|6h?a#{OMTib5(UUpWC|UPI>Q+Hm~6* zcs}OpL92x;L8JXab&4-#Jf#R)(!dt0WBZR;%H~&|5PlFYk+#VkVB@`sSfpB(V?cGC z&k)SZ4LO74Ek{=(h`;n<2+guel%kC6L-?2@?i)6QH3%ebfH}0;3kT91Xw_AQcu_@H zEz9vxW<$oK(D&jH0!}oh5UE>l%sVlrlg}B#>kL7=h_6|-TY`9oT<^JiKQM~rv#}Q^h9a`d+^faS&p+XnM0(jRb z?`QIQ1@U5P)R!HK4MI!oUrXXNf}KM+wr#bSZd0XUiLlfF+iu)srxP!o+JSI_Xg3)% zH)(P7Lg~WZ_|RW;SMkk}g$>)#E#r3^o)rZ1+E88lTTF|Nb{Z^AVWSk|mKycU;J97F zA-<<(AWK+|OIIqExG$>k5keEUgDT#X_l+NsR3wkrGVof}yL4r!Vn@5fgm#Q<;H(iZ zW4KqmmEfMeu;+S4Vhdx-^&lqL9<3jqYwVLeRVgai-4tKHZ}AxCK?W(46frw*sp##$ z=0NjCRwXtEaBRC-!?)juP%d+RuR7)?iu2E-H;~@Aqx=I_#9vcoVl)u5D_uz^LBjAj z%T?JxC`dH63zlRS3*vROUFazgJ8?D(p|Yr~W8}3~Dwb|v@y0$GCtXCPma<@Wy!bx7 z~B*QptJ_#Y|LpgztXNypxsiHhPT!nxe267*)92%8PlSX&`HSb`n1lfy!c z?Fmi0;^x%Y|B&~Hv$Y8J*7P+)v>6qUl8W=L;B|#cU7l)3Q#e{zee?XBNA-BZ7b*yO zQAY1%$Ev*xglAXVJajzDR9k`X1;JYGhw6mR(k?s(Ga973*z!YaOYkXJ>c-9(qw>yf zy3{y3QgQG(0EjigoT+23LqAs%by5_RDB)iFOE3@N=)42~g2=FW-}^sM=-YJCtJ7%JC1Ob5A!Tw(%ytL$b#c z0$WFjl`%=~A`CZbPBSp=qHJF54a$Yc!p_uIuj72m;R)S;jlVlFJt$jg6VlJfQ#}}- zTRN+%p($Jpfm2ROM7>Epakzptz z;IX=?6`X<yYja z@<0yw%&AkrdHxdIw(M6p%mu3#CyfXu`Z&w~>i6yzSt_SNo7|tK6O^jGBRwwJg9Z)> zDk%@v)ago-g%63Az>n+8755Jz&5fJ6VHjoD<2yYd;J|JV7353{emTAujbaKg(@kFF zzJ$U6zt+U?2Gz97PIv4<*cj7mTgiP=Xlteb*IsV$H0Ofuafv6!1Q2^Km;7!O2*H7j zvYnPrxTqm)kYG0YmdT@Mm{~RL8FyZ*B5!-lGj4*-GmzxcR_FKPeWIXQGuhKJRJ4Jd z84=G5-W*cQn@ZqC>e=(^6^ZCoVHZ@qg^)AjxNxZ2MKnu*OKg+kADYWc5`eQiFt_2e z^&oDf4L9dEDp3kP>rb!6Jh7QT^JW?`oCFMeGMBRkGl{|F9=GhkOgw|%A>@v=@SfGP z2k}&7&d5zQb~%PiRLrkaxP405ObNjeCJ#eK^-FhiIuN{xMkr;RJDGqvNA_oGPC((a zl22aPaQB6jG7>-mKLn+7`{8XYS5^IJ zZd5<_5-cmuE4>WN)4AO0BWGitGGdrjT z&+~kscxouJ0&@;IqgK>|F>n;=7MOHMgb?tB)&RmyjuwE~EKo1Un8(TyHiHX9&Pvm{ zT-LREWM_t|BhviJL+}}Wxm1`){kb-?wJ=5_6qZ|Dry3iv%3BvvnV8=l>Prxg-&D0! zI~nWjse&+&OK*LJ8;6WmuwN-+SjgYt%wBZJ^I-|;gMcK`GPFZucIw&t?M1Nq_-X9k zAv|?BcsoqWlp%a#Lcj|%^ZK#>b0?&^3y0N%A2DcP>K5Z4Q2n`g5{c}!QZ4l%d=Df~ zkUZ(f;3?}P;aQ4Y?xS72cNoHy9~U)K+^blE=Mkb-`thy=oOj_`C!)D;;MMrh4rzt- zxLCmV_2q)+mh}I>@7v(Pm($1I@)vrOx2NU}9pE;T7t0Qrn|JliGY2!x3GNeApcg}* zdZ>8I7TpbXE9GpP~<20JYZ(8UTMOYIYB4677{*^BT>zj|w?TiSJ{ zKe9xsISjKiJ&u9}rq+&qoTR~(hUrkB>Yz6BJzGw~ZkeD#w-#F(hA4g-psN&SzZqEm ziw{dgmm^s3Y{XzRIreB_X_fEYxPs+#vkrsUaq64iw_x;b_r-HC!Nyhk07&pLb2A4# z%s_7N?%W`s%xTRB;YM5j_2#g&ZKbX4gxZAGi23=NMiHtHqZsZOnH%2=rY3;2Y!PD3xnyWBO}Z zfMR>rx`*0k#YuXIt^m-+l#XwWe#r6fS6kS z<*Zgy-hRwU_oc;Ec_&}2Dp?L7!>T@*;KWP9R&K7M3A5|Hv`VJ0q?`CXHQMpPV_q(T zjmhL=ASjmB;5E+?L4)+si^<17xeR8zdlD9|V0Qh}gp>HuZV!UkEnT*l-2~~PJWUkv`WY}=1bS&>rD`}!;Ix8T z7=QTUh+XpXl0tg2z~qLIGo|Q~X5CnhmJCy1FvA0w+92KI$`xSvA%KD;LqtYEeyUuw zr*?PqTwLlAnVZ7qM|pWZ`!=NlT#<&6Rltb%Fl$Uobt7*g zpjX?n+@RYf6&sUFuT5wrj|iVgDA@b;#5oOWvw)q zHZ$uJ6k3)k?;zU4v>U`rMn-!r`Fz;?QXaXHOdHIvI(nw4Hb?Okw?2PphuX!D9+> zk}L&->7?p*BwTMiRWe5-01Mr&tbSNeW9_3`d2JUj zesFtW_dS@yHn&O`n3$Y6A&(EUIbL$A#$?G)A548(mP%4YZ)T6wqA`c9UcC6Y;vHvp z0n=LifiD0s(9$Y*7okTevI-LYe=)b)*lDO=T-0A&Fu}=)0Lc~O4P$E?cuKA1F)uU_Eu4qb%5axxYo8>*>Zd4Fdz!EbH6 zYOU(`H^?7I!k5J*v>Ms*2+zGi2pbm>n?=QGd5Y~?Nq?0WwScB8F5Bf*wrrUXAUTK&Dy@JQqODN_le{NW=S5f>#%1y=bx>jVQ(Y)w!EjZ-T{ zsd#l3{26r^;Nx{L6i`^&8yH<8zo%hgheWDSD*E&J31x8bO z2Z~uAFaCW4yXE?dP-ZGc9Y6pq_+ABkeSf#x!4*V?aU91(YlVT(vSq;6m)9mt<0Vhj za_3?*us{;0%Kxq*hqDI24}TKCjsjovY;&mp^m6|B9cOGmoe_{rN!{U8a^x}}3v)aG zaDjP4r1B?b2za$_y~=0|Uw8o2Jo{{U?LnMi@zpuN&4CyI@~%M7@K3PLEv>m5&hg|^ zGJpmMHC1%FQj|qp&!3g*r}Zi}0W7y*LxPH#d0QT6P9}^-JdGgjX^ zm7ZAlL6KcQ;@lG-Z6I>#}YB=#?KotCJTO z4SJ+3A7AMaxy&HhR$`9f0G7*-6>|buegV+bq+RmPol<-Smg^wG@_)oxjjy3kOe~3X zQ0)12&i&_@31Fw}fx3;KxT@m)ib!#K{akQP!~!Py(e2oW7_oT_5^-N+dqLTC+sEym zB&-Ai#DYs(dtW&s#3J%Tz+34m%^#fzP_*{_(Rr95XNG^+GHxO<`KRbLKf1p~fEw9* zHs=SoD}Pez`r+F3T{50P)nosU&3~7MIwE{`+xKkKF8r+>vHGiw@7f9iMcP$gJ3;C6 zcW{><8-_gJd9YO)7C3M~W6{@p}^GICIW&sZD#9x z&WFAH*zOZ|w;tKgk7C*9lD`wZvj^Tuix+v!FtGmqfI8!(PV{ofS@3rcF-I2yPxgRc z-1%c`R`Zp^E}YDFWX3!npLrz>)9htIG84F$crz~8`FMfVch^INuU*U!!1=Lm2fq#T z^K@JD#0W5g2&P?HbzXS>;Auui1m3XOrN3pdiBNt?M-4IGBSFL0ErcEOhOr$VQB`0x z&%#tDPx8c}YZpI?s=3utY9A++CV_h?Q1dgwNik8oZB5UuQ z!vO9GDvRet+}v_!bO8H-KBQ| zZNZ1Ne*uo~U+nlN#qwQH6I43%+W&;O*;j!ARMAbo_}I0a5B#gQem>}PMh=C{^AQcu|1-+B zUSeis?)(&LzQjeSQ!jg9{mt{{Yk&%n>$drM+?XHm@=%G1wyZ5#W+Lt{9kA`<{dge)FhYf z&Vc!HmtWKqaiLp+gz&`P>+5VLaMfiUKB)1(} zsHgu6Q1JXi+52CB%)KAF8~*}t`JcIf3KEb8P&r`Q%-hY{!z~B++H>j8Ke_OW&ON`1 z2<@_y0Ftf+UYKW_ORFU}erO8)DHHSWZL_^ynVC>to_Q$B4v-=Lm<4|Ta*sSjK;$Uu z$ir*)d|mX9iEJ;RsJKEG=BwX-?iM9AV74<9K^aF81=rO-{Y@0eRhMP+>*#+TM$Uay z{R`BsG4h=xYX#67!NEf`b?$5a{G(R*AmikFe@4Xyys{AcHp4L6hv==;xdvvNBRl^h zUf$gLF`roo3)zw1s!Sk{oFowa14Vds);|_!R({MS7Xqo1)i=4*T!#AyoGsU5WCw?9 zPm|QxaQp0>@E-~s0A(zMoaKOK$dtDBZa?zf9C%kUmYeC@2)AQE@J|z0W^d<`*2If{ z4rA@@e-xEwoSV*oH0(YUp zw}JbSBXI9au`!l)mJ`uTve}O6H@tQ$rD4va0eL}C;W=Ca5{WH&?fmS*H1KuvvLzS6{Bj=U*Lh0%8<++Xl6mm+n=7{nKpLYh`>`%21LWy}TMg#Yl!fLCun;~CF4P6q zUV3_(-!%qu`0r;n3v{!~E_wbyprz!I&d+n2!5`>38xi_>O2ls<%@+n2$_K~R%*Q>z zgZ`3eZV=S60h1IhpO2D2Z^`T$K+{txA4`=+{TvpGeNvO8F&MfDwNb0P{ue1?B@-sn|7@3LRDza;fm(LJ!iN%?eL@2I$z*l5KRCa%8Bm)5@_Bv7n%MocD&kQ$Szff4eOz*|~w0wsoji24t^@bkmc zQzDZ5&_ZncNIz#)79*nhrG|@hB ztNn*~{&KL-(Lc8hU*QLkL*_>N?{1qfM9oQ-#NXyqp>JmZWV(9+xB)veXToE19r~{c z_LS(2p@p~qlE**r7=X`@TVY_4=T zw;%C+Y2QY>f7@n!iP{AR2ZGs~;BVP{-f94*fl}UIsgzGa`p>5Qmrgt9U32C&4~pRn z=s8vO0iNH${G0OYm((Wr)1OUyu0-&soV5Ubejucrn3GapM4NxwJ3mZ&^;h)pC4fE_ zPHo9L-%68zmZ3h8%2(_-7i~V`9@x(XXO*wi`%il6nq{8>Xn|PyVuA6ktoc)~`>Y)F z=X=c|0_iyamdJ%#^$$V=XdO%++gAZRxa0#m7qr&r^Zwi9Ij`}68yOa|=Wht?`(~L7 z&CrM62)=*LG9PY;Sm<&5AX?{QZ}-0#upjR@SZI~}Ac^pyJ5ls6=8=zgdVFj~?EQ0^ z`o!)0c=K<7OZGrBYB|YIE9Ytj3yM8nSj;I|1c)Uq+vN=*U$3m zk2%bDs?;A8>*x8@cl`ZtkWWB2aF;4ZoQ2WLX2YNTzvr~8fae7S$x;g+trJLAG8N4OmE;;wWxE{ORRI2ZKpo(uOQk~ya%es;|BXmog zWj_C&ucnMb;%WcFkz}lYGj|X%YAG8HA9GjvEz_P5o24l!nqZhyj$3DJCsCK#I%|f= z?eFS!{_<3p6s*Cw=U5mK^rJ%MzZ@s`f(S+i{(_Mc2j4EuY7Fl)*0I=Gg>xTHP8iqd z9vy3oq}*cf-h}UJMB@bDA>idrAc%E}-%7C+y~xwp=F> zmQM&n8uqv~fHzK_8M@SQ?pj6Y2A(;t-vE!Z*2fZ@3SX=pt{^gWh^4qiznyVfJ!JW`f#Hj>BHgJ|kWCfsYPz z6JN&cDz|QwLz}M5#@$ek%;L5+PSGCh1}PbkuVa{5DXHwilkj1;WLA44+fKH>0iDPl z&%^a%ZU>!uT0GIL1bj@b;U{6P4^zNiFv|wG&0Af^0Y6KRiaQ_sGDK_oCZ#Yq6ju6L z`aLE_V`_}#%NRctLfuvYd0(9@*ZY*Bj~lU2OcF?)#SPgQ;yR;EJ3$*P3~_o=sS)j4 z?fl=I*MH?_X&B{hmHZ$F+39%l7} zb|@8PU}dYg$UR!=`67MBv$B5OqwMWdj@cOAd&+_JCdIl4jww6r=mA}csW|vGc*wZ* zg;&aNO(2}qmXrbrwK=BjbWA2)$%*pFdY@N;*>wX`dLrjH2B+MFi<0N=H9>NB2Uxts zBYRn)xuiCxaas9USLNVF;`^+AcFX2kcRwyEi_1>ro=Zu+75D%Icl60}EDdxH;daS- z`i)d+qf@ySt|22;_9B)K%6;u4GTLxi3A4|WGMkJ;M`xn^-#BIJ7R^3$K=qWNh=E3w zhk*|;zE_s8-b-(`$veR+HWqLPKJf2t!$@v&Rn3s&#FL)z4pz@@e-6on+WT9qox-bv z<0qS2{Ss!^`{e;v13|j293exlL0b%FGo8vCH7XTPWZ z!YkqSz(Qj3O*WIGj#(|U+t!jM#)@@nXO%fJPUWvmIJ4eq1}f$4g>3_Dp{e&0 zlv@Rs9T|OUGSq=~H#C-Lspj|Gof$aH26o1z*`y(hp=dPn@Wi&lw%M#0@4dl(g>AI< z>0WgtY$(jaxy1L1ASVZ`U3xZ6P=~cr8K3P4%fO}1YHoM3Ij7v(HpJfOCPwm5Eyw`Yr<{WfP=~vhoSag$d4=j2U{lLz z4^@M0n@w;zE>c7ZWPsO*AF{Ku6gSj9hDsOGi@A`lX0$!CFSYVzMi#BDTN}<=hBTET zInq^_7o7b@n3o5K!Xrt5?Q$Lptl_mwqt5IJ-p?N0jdLw*ih>YkB$pzhz555ncJXkCBB2>xl? zM`fOd3}2RB&(X{yA%!+u3)zF6eX2)wy2yjBs!gV~v))|_P5rLtj7(dnI-p)f-TPEh zKpi#@pcU;IBxpdyKAv$D4tLDGSGL=e{`h@vu*sgrT15@V1Z!NEPBv1wO~#n3-OdP~BCVxZXF@9uXOeR9 z?ww+-Z64e+@GM87Gb#y%X}OgYYLc89HDzgre!$9qllV-g*xmo0yI)pjRzr@2mJew< z>X4yvWn$=W&kao*vSQR?3r}bp?M?}@2_i0YDt;uTD8aC4x^)2{$fR){IPb7PYv;i4 z(@?J_{FtF>WfB(b9HNEhSoj_@L3ZP=yC)dOin}}UR#Gw)HzMz46npz6oXhI>L-o0C z13PZ*W8kzLy#dDZj9Dqd@+?@~+l48hg5XT3B;#4gR4{_%#k0 zS!DGM<1ySNwtlKIP6xGE9TfvnvZ__SvJx**UHeo@eTm0N{5*#6$+*0MAqVAPQ$+gLw{&z`6#MROi~cc7G-(KMetVqaBtsv_?Z z*0)-wj+XDVioLc=r9#1pQ3W{LagJ+-TdvxK^99eAg^lJ2%t zOp`Pszt+3hpnI#af;niD3^3r*o_NVymjrv%p|@l;T)=&)k%S$f4Y5;VlT zkmVVll!CZ(OG)YWv!li11I#A-*0zE6sP^%d7qXQ3U~6?!ynSJ(#&mw;=;}nh^-{Q+^qu*O11AMBF+9aa1WfR@nOR35q*OZMKn`ZB(L)Hs(wRKffzN}Y(jx@XQ8o90I6pUDNF)XH^dISQ`q5= zs7GoRWcBO9ZjY-P?CvSU-^SMFJgkYK)um0Tjq}a_;PomYStX5WPkWHI#}mD*SNUe`wK@dtSMw25_$}bEofvo2G_%cmDj0iDF1WV>JJQNK$7Y`! zkkb;(r7{h9v^VFJ@LR3XTqh_=@wN6_n?0#a*W$nzT?95xZYwb+px^gButM@X_5@R4 z+tf)ow?t_(CuwA{&|b(isTqQxLUC^(qEjbfBfvwWgo_pQ=nrDN{j!|0a5)c?i~FTd z*R$KyDr%Y_mC?R_q-Im6OzQ|{6iFo|$ib%xsRc(3L#Gh!mtAg7v!tJ0+DlAS5-_W20RFqot<#YLJP%Ge_zOvYKcX}Hpa72GStMeCS6@-|n3ZmXw#2=`x z0xg>RVAu61J0n6?5;ipnE2@H!brykl)|j%xzY&q&6PEwfYeZ?TY{$E1`1|Velg^{| z^P+c7J`dL>E1k^jlF3g8p6V7-jIn!Q41DlS7}ivhM})7he0wrA8exzz>|=Ol?{xpG z+Yd~0khcXQiiQ*1s_vMG`f5H?NPN($NgA56Y6TVZ_Y28P6+o zUp3Yva1W>Z9F$#`F)Q7^3?4cH-C^Y^f*Dp6AH*Nf0KK`=X24FCW%^siE-^Cgd#HRD z?y`N>#S*!e_lI#pBiFBWs%@frY(|^h>`w2Y8qr@mg3 zT^jr(T3I9Lbz-{DG0#U4@cw7K#YyDK8z3dnl8_nFfK9o+Y1bb2{&P|Zl);V!!9CjI zU=z0BF->T1v$9~Yuf!0_xk6MA@?KJoQ+QZg7lR#_SLw-f)hG+EENnaE@3HxfDktoK z-{o38#(wzN(SkVl-V#Y*ss$KVBXmQmu`c7SexQrD{Nxm3mc{0 z`XWK71QTS-R4-pawuYgHE=7BDo0XD)H+f~eXK*lMAnt|ydw*Mhga$HApwi+6x-GKl z3C}x_D(o&(>_x{u<7sBP_4)DetZ;VZ9~Rhi!HNApU00|(yR{A+@`R_xOU7`Xv>V;# zDE|%~#XT}kS;H~C6U8eUYn=%}+t{f=aMhlDm<;1}7ElXNTRz>AJ@U6+{c@6mtDpL6t8B4is33q&BW`BCOk}j_P!pSSj9G zQIV_pc%4uyOvE2E%0yBpN71>DwXtLCSbx0a)HK?InQJMKv^BRQ#fVCD&PA#VdyeR) zh6~sm=xvbF3B7YWJ)&6Dx&6_&On}mmo%M7?C$&aqHy4PgmM#`Gvuee{n9mEl{LD_d5X{>tYLG4M$*rpCQt zyB|poRsT3}y$+;qah;y4R|`xl)>@ocOaVLbt+{N!u3;! zZMP0^yxN@@Owl5+<(Ltplq^_UzWg9&UE7mrMMWWabc=xr^$CXCSX-#L{&~%j4dYS# z^-_mv0L_9t&DL{ z;tdzrs60C8b=~4c^MTPDR(AfVYX+J+@TqDz>YCtMv^feA!&zc8CZDoOw+vl7>T^$y zGK~fXjlHJ$`-~LQ2RD_QqPU5mtyy@3*#sU{Y@HF&Pm$2NtlsDsPkRb~9X6g2%=H}` zdMgMOFGd%D=|ux_UC`emf}Oe34{zMt%(ko7$A}aW7AImbR-GehNOQKDJkqpFBT{JzQ&~eZJmf&w(Ie^e12N0U=M-i!@ zoVr%qU|clc=PFNcQFuCW-q|*{MQY|8Ub^u1H6av)F46){J$;pI)Wed=ZzWx&iIh*B zR72>ID5L%;lAOW~h+1&9_A23{>!UqrErp5>iaNO^hYElti$_USmIh6>@^Z>`U#2O~ zZ98l`p>62&N~p8j31PMX4|s^}63b|7slmb$UR+N$)BttYILQAe5}dX%HV#ws%s#AH z#a=M8RjmcW%!pJDbf#^z(|tIgy-snrQL+1RRMgEY+fs}Pg=0;oMDS>)mzTbllak|p zxC|-tWNe%!@Yvh65gNf4WFhiR5%9@i>JsASBy)917@@4yQ}y6vB|FTew!h_NjWx0P zRzqdU4Z9^)_!h=edOYejC>bW=;@~Qm9;{S}9oJr$;1j}a!d&)3iQ8=9=|86(cpl?1 z*&>18+Awi6v?xXk5{1EIo9_;9Acch#2Ku=7ug3U`uqc8Yxo0E3o+82+?YzMaQhq8V z>T_vEjb#(sjve1Mqi3^Wj3gKO-lx|hWplB7im{f5n-@$=A0^IlAGbo9Q#w-Dk?>iM zJuZ7P_F6YLPXs#eMc7P7DeXONWalYJcbdeINaU!%c)~6d&hB-=&F-WDgfH^l3CtQ_ zA;`)qi?SF46RweN6Qysw#jvVlz#oh(6ki{`$9QSIANLApQ1(4(OVg6vto*t->+9#$ zu_d%#YW`;=HAH>LUq((1Dk%|om!7<+c369^AuCAA_`o`)aOj%?b!iNFA! zo1UW-6GMvVe8k)efB8y~&oR1!vv4u<3d-)8egFLz9K8_FJZ`3fyn%B!ly+ z*Sjnx)R6=Vw9B`IpDKxvS#K;LE8ShcJ3UFMnZK5(q*)}N9A9n zRo85VQKXB!MT5!0YuSwOGi;ohEYoH^r$gJ%(b_t+m(pO7;^CEmu_f>H7OQm328>QAq-$({ywA_Ot-o9Fd(dSGhp<@d>^?<)*P|7hO zBMV{Wdsv(!bS!iKNmBDJ10804lxe~kv7oX?K*rNQfY}Tyz$@<_ty`)&jY~#IXYhi4 zTO3+_mKOZTI(8B&Eol{2RE{#}LSsg^2cyKT{@}(ZVH^uuTi`T3P|0ZO>DZu&SK(sR z5Ggf-sH9L3+%i3tWK;D*RV${+0rCQZ2%|-iTF;^g zje>rcyPgOe7HdYzwehMwiPG(}%3L<**(^DFSfn-ax_3@Q(7s#ud~za`J3KUxs>uThu~2YfTwgf(w?Pnj4@+R1 zL33lLUPm*@N!Oq*JZKFoY*tqRZr#2SewN}=E3i>I*`XzIboDEi47E!^8G%2Ql^>J- zg{%gii~G%M-TorEBe7=x?%d#exd!l1H+ax5x_)tAmHwF%j)rfufTut;^MHr`4zf7j z8N(+>f(e+CpQN;2e3g8K1}O92m407+Cx_{f487lWI5j$dAp?3c?x=#;r9#jDEGzma zUrRP34B*4e>m;NY-&!pcFAXE$mmK4|GmB;Nc^n2)Pmq={p+}s->%O+}M{{qnRS?Nm zd%x^J_(SYoy=#ze;x2Al&I$A4Lfz^>$JlqJGLh<%HTc*~+j-2G%`A{o9Q)8QoOf&H z@9In8acqelf=&aaERs`5;S@Y`y8*fI#HIvjGMlJCwq|ym31Ht2Z9lDkkXRqj9?|ha z1(A(U`*VGaoo3G|Q})7i+32_i-AkzS&8$qkUQ;1XG!m|Rjr9n#%-+J;XIRfN#v!X^ ziUIGr@2P0tUALx7?{%}OU{-(E>1--@8kvzvjBsv>Q8ExJjsxkVI)Dfw6-4s%oxNt)|ac^_>| z&GoKy5n=E;&vo1{F6}lXv|w_EWgk=VTiP%>Iq<;ayp`wZith$9TaBy6qp!fvfed?u)iC4wEa$u#n|*mu?>u{T0`~-_Fb;pwv#D$1>;d z8#`jg`_;hO4QEZ1$SYq4$Uy#TA@_nUwzx}k3bUj1&X2ZUp--9>ug`cB7ZW|+b$Xml ze`N(~h_woEO}J)D`f&5T$!js^lBZ8p9`=cNIiK%I1)u0XTK0lj4c!xbb9k&Fl52Z! zrv7HgBPJ==^IYa%QkDENCycg=5@SFBxAir5<>#RvPxk!nI_Dq~zPQ z=z;XHYuyp>=*$XQ?!nz-=@4UvX8S;wgPhSD^|EqO$Eaz?8|(aP@9s5SPY7%2q!@u# zF~+20@1-(oecRKQiQ=Z;dJtD>SaO zx;T0vU5@pRBlozTWVu_)j;DA#TA}8woC-~KwR_p1#wMQ(&G?eBC^AxYnSnXcn^}?s{0`L&Cy4a@v0I{ zXSxm3F}$Hdw{ujV%j3Vkl$W%H=fp-=7q#MBEaFGcXU zQs_h?|xO2 z{Pl$7882^nNdn;Mp+!S6ucfdzht7?tj|x{faf_lt2q|5_A zMqN~|eudgKkT3Ke}F3q7SQ5l^QAL2Y>6RNWL+qqIseB z$ZZQvNe(5pPX*yC;MLB;Bd6cQRq72xn0mnTKNq{|N82k<@dV}n-tN8oE53xK|*6-%04 z$#!aD2JSJK?Y_btAITgrFEhhL;s+>+*xG$Bn~o+;dg~ZwO+ofpIBO%uHiH2Rtc}>$ zQ`RddX%S#ul+aW^BRI6}g7eG}=b_`Uwu<;#O=OUoDz~e=Q8+)6*{h-ezuz2;(C3Fi9+UsjZI`4mm8Fue&BD0?bo?bemTI&pMTWOJEL?1*`+EDftra zT{re{l3NG=*;%VX6=qk6clJEb&FTPMjIWQUPc~1~j%A~dzKN6kt>i12iz0!2_=z9u)TPHFCN5W760O*t3%IRQNpDP+8ZpfLs zpE$jYOJG;Uv~d%|-B@Pv?X<`}1vD&^`s``4CItAi4M5kCdUvDzCR>IN+c$9&;ms5w zjggy_HPv?hI5v?6$_z@h8V)H`U=9Qup^@^N2rH1CFa}X#_=s{aWVH3oIBIlNfei(f z@DtJY4QL#6RX69t+xhjDtte9>q=c{@PA`EUWahjaN_Gzdzp!!fJnzYFpsWB`>tSzi zos(jUeEOkD$a+X^xM?I%llXN&TO}FGVCwXh!e8ivm1DB(v^_snGw(9zf+{QYC=3Tl$ya$OIkm&?xADLza->Qu#2H&Htdup~m z+JhLY=z@7B%-ngF;6^+L!f4Mt0b5_8kMtgS{dUu%=R=+$s|`kuuC*CX$dn48?wnbQ zv~+Y;QWi zUhm^fcPpK0vpS+bm3rqOFJMC{{V2Y3a*tGK;N+!*N58My;m#cxEQ&LM`rCL}EH#jm zJe_ihs$4^`hWU8%iqeOM$Frm#S0orb(^H2~3x=C*Mm4&XdR-@%%Xd??>!w>a!`cC1 z90|$Blz74ocW0Dkb!OJDH|4U4a+SpQLzaNYIjHs@TrVN?X29yQWr&?7k?ORZ)MV<} z&a_mkjA%AZ6Wm9GDLa(q&eSb89M7sF4N9d-Hlce0AC#X&Lcmjv@zC`rqIZ|R5sWB# zCS5ez363ZDKZGf;kx!!Y;d_qJuhrfg zQmi=)6wkAy+;ncH(~ka%I_>47U{G2D4@7(V-6@Q7$G~FR!7mG`rYM+TGT1~Dq3InI z-XK=-KC8uIDkA78C#+h&PLEXLGZJllK~dvi@3Cj9n^yVl1V@3&K>FdY1& z7$x{kK+hP4wE9&8H6QA!Sf07#&c2rlAuvkF=#Fnwi2B=De1@6`j`iTCf%O%Aqc-w} z@`YQQM%?{&BTMSG=N?R#8TPsULGAmheJ}m`SHA|_uEV4Kf5qKxOjB1F0N`wBy3Ilz z5Fvs#3|7`nLD|3$Xd5RVBG4{05s^iZs>lW_Y#AagFf^;H83y^pyG$;so^n4 zl}GSB&XUQHu(oF$xs_NZDE`J95>UmM?PAqFZWB%7o^a_Cb&3oh>yVdLj*ofbu~kM} zP}b9w-7G{~_pq5zm_B6HAE!WJI1$?$@$DVM1gc2f$O~Wt&6Q@X7_`%uk58^%2Pdlr zj3`e?C|qqoTg8?j)n^v^hrmc+Lii%kphesLURsSj+yiip$bF|bzW)Kh$HGMGpO5qQ ztFW>@i~2tD9f?F5Vm^yZMnJc!i?tL#tCvGNgu3UCX*8QVDxzjYYp2+n%g7}A=IX?y zOh>1uGx$lu3r1k|1Z*The6uMuj22=hg1nbyxHoK$pmz-FeMZSXNeh#PCXnGa;aqpX5Hc zN63+6&rAm#DsQLc7@;eh&HiQLMY|8QvGRn^t*T^`iSpBSY^_~I=|Y8>@P$o9{v~e) zrbTcYNu<&^Wz$tbd)au%^SVVmAl?9bA=}0A+(fSC$VC<6Hw+VrIOoN$t{Cdg!XByl zBlm%s%{@&jcRv~Y!Llric?n5uAKnWjUq56{?|ZNoKSX)|Xnzd%oDipxiPvTp?65M3 zRe=4eIe`t>!dB>cKKxv>Ey5GQ+?nzcYwsXPhx{%(h6Dpi7%F3V8vD`^VhE_&$T(bdNVz9pA?eQ9pb`L;uq9(_ ziiUqr_w;qFOUrZJ(1FUJsLF>xMlL&-H3HbPoWh7&+uQd$x%=Z6jBTY?{V$QZ+L7D* zIvd9>xAC`CC?sb+(pV6Qi$^jWlQl~o^%l?9X^r}sy1&Tvni$JIwj_8-^ioI@0rQjU+PlK zC%tKu0%FSRkn^y+!(5kth;)2%cx-?=gLc{BdB@~_lC@vfy_Yg{W^@-PWwLVA4}HrW zIW2qK&>Qt5!+Uzeqe2JSmGBZ(>^tuua6{cAUVaJh*6dW)%-D}rmRlOY)gVlmC@w`~ z*^PF}{ydvK+pne=Zgy5P$5K%O7T?P=O(o@mA`~dS7-~ME;i;J8X+Z`iN^Aq`wtB9h z);_(mk!Orv`Z{(Ucw+uvBYzOhU*o_PA?~zt`eEWJ8j3tsJ6NEU7OlYUDHPXdi%wm> Srx{oi(-Y&Uu^pdgo&N_@G?h01 diff --git a/arrow-data-source/docs/image/decision_support_bench1_result_in_total.png b/arrow-data-source/docs/image/decision_support_bench1_result_in_total.png deleted file mode 100644 index 9674abc9a6249a91086c12067ef2c0993c689363..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 10412 zcmeHt2T)V%x+on*nu;R5*+J={_a+JvL6Kgin$QE%tDsU9X)cf|h%|xF1OWvp;-a?% z0RauYhTd`)TlbcI-Z}TadGluOow+A7$y)!nzFzR3IWMjy|?)P7bZVXldM5fv(U?Pfri} z#A2~TvM!2>iu(HcMnKQNz<{BlAqImXl6F>5P%sNhh2RD+`XIQ0fdP@U3q&F>FF!aq z2rVHFyoRAfQqB$y06qN+uHhOFEe(Obdm-uGarw`4Dl=*aimE?BoHx6bJRm%AH*L5M5k04X zfS{0syu60iEgd~0Gc$yPlarIzqsLDoqNAUurf0t{Dk`dYU)Rvs+A%mdGCDT?V|rn6 ze8ND^afxT$sh}xd9yX5)}bY$>7P84knv7I0NA5c z#G1i&az>BX1t6iU#u?rqL@4>dho^Vt}SzeN~`$v!A((|2ajLr9}M>X|<$-FS% zf%ys#SH6}>KEScsLDZ1X8L7xu-Z6ujIeAlmHckAVC?FHk@9vA{vBQh%EqsL2r`PTD z?ehTx!#oB(7FRl!on@wS3#+9pT($R>6Wh^SR4px)ru_Ftr`mT$h*$W4)aZb7gC1hG zEbd}9jO0)427ap+lb2c&1{BohX`3GLj zJqRqt;9i@m6;ua2L073!JVX00%&o0>+jipA07;(&usg@y^9XX?Ml2ECE+x~jPVW&- zJ@jt&G)t1+qSL+7c?XB`VUzV$7asmkJ7EHSvetNQSow3(%m)J)#H*8SVQIl5mt6rh zCQ=&TJBg#yRLiz*!^5D}^J==;k?YvZkSnVQlNjsrO1jyw`ztSyVh$ExQ%1=1uCB@7 zC1(?O%?S@ww z4}{5Jm*>4a?Sgk#HDt25Eb_DHDlEW}ZRerkT)$Z~&qrIouegFa-E*uApcPNFA*L9X z``l3FPL-jfYho@{bbQ^y0xA$8SjcO>V#{!=<}Q8d?J7|t71na=oBiHL2j4JtxtZ&I zutLh#82WG~`cqp{!Z_A1#CY2e4E3D)gvwazs+BXaJpLKOXDm1<)6LbrGbM|;l+kK2 zI&F5Lx)H5aaJTf6MY&(i$@ePqC%;vK-K95}Ow)#ZD+VrNcqOx^3h2U^@)f_M<+r>?52xIg^)&5PsxeWGXzG_6pTA#1p+*mHKimFU<(sPaGk|4d~8Oydj4OFxu zPoyiDT7-qDyv<5w=w(*tQSe2YSC^yQl3kW;NhBTPTN9iFtT~1n2kiOeC4g56&tBSf ze4TxLk@j<}@w?0E!EPn`(bFWsDK1~FkE<*^@28vOr>H)jyT>EZXPv}NIcHEKI4I3< zD_)sG7~4R#f{4#Csj+$spJBzgoQZzuCiO-f>)gSvX;!#sJCDjNF{#@fJRdcn7vchI zgU?;CSmCy^d~(r`_svwI5cA1UDMVd&LV0k?N?);lKND@d;ycm#MFe0)S%_*L6a2)I zI>g7nZc8JP&pGVd;=~iST0z~fo^leW8`v35av*BrlZ=P4i09%)8RWA}il`R5^ijkvXEJe$se4F|CqCG6 zx_8oRUN6$>qHQT$*262ynI|lu>wc0;W8ls0>!U%y6|sK6E{q)SXk{Wc)m037J-OXM zIpq8OoQ~WuT2E$XzGd|=uvA|Dt%+sIsBX&BL1mi#p!mjHg4aONWUF{ z!HKX_E!)xft)<=w zn9Pgy{425V3qvMVd~WhZ-048}pCklQdb5tA^-nL(gGHIJ<^qf`1Kn9$B3jX~}nwbz^hlqCsKy?L2>)zJuZ^x&hRQpQNVAR`GQvGsQINEcH> zgMbWJ$LV$26*y(kqg5?0`=LpZ^&u&5m9N>;9}89o7X0?jN`ov3vDdvZvc^wMKFqr^ zN3(dp;M?(>S4s}Xl49h{_E1IOT^1jBuc8G&}-^`zo7i!Smwp!d#f#er7>5qeGPg$ruVit z)uL7P2b%?9dQQB+0BDQ9HX3dJwYIw$6E2eRY?h~|yeog&tJB&=f19yW%n--mfCPki~m;WbATwW$Y;j9lp zT_u3t!t%{EY_2EyyO`VJ%c=$(Q}kRA>j5xRV19{^{w!P(Nby_`re=I$72$6ZbOen_ zvLU#BQ-smg@dprL=ctjj+-T1L4XSYhMogEkx^QTWFt#DbA$Ux#-YJ+cm|A;;bGL@P zR-9}1`ll3ML3x4|vsh)}sF>4c0p%W@`L!6o3xU29PyD8Hd{v^B=2H_MtaajujdMoF zGvgEMH=j`%&HBr7`(dbS-Ol-{9}8_8qcf^86V4b%PKOCt#5fR{rdmoiYfdMiJ8=^b zX^nw-;eSyE_zcCf>C;g!BozatGn z-}@Bi`ce&gQje6Wxqj@pQR8KcC-S^gL6DK##3?EiPCvYM8eVYao8>NX=u$GNiM(6i!oEO&2 zrF-O9cYOo@cp*QHEs-_nq7Jz z%eXnmg5RdfzG~*$f@WDX`BLet>12wmQ+rA6Q zOTCdnW{`8}FU(6k4QLvh-Fmo0z7CafVN$L0T(nCpF_b4?)IBOntETc~AIsSnv_O8| zq&(AD)cgsN+hee%u>ji*JjSHcKqsVZCal|o2i>g~0O80b*o|$4FiSc9<(YJlR9X6# z_7NrY=2m4~D}E5SNlNX!)RY1n`i6lrV@mg9DQV*+Rf8)=d)eT#_3n-v^noxsdZb%E zHAgXhKljSSFv8J*w$_S;19$BmCg*ZIcb4aA*_`A4-!cda6fx~;ZQlw~Nsewjm1;&= zXl?Udk|iv!{Nz#}zL|R#XjO(hVC(t=$~APA3zR8XBMG!;{fDr=T95|$^2~oycmH`+ z{{3mP9bU1{skwIktj}1qpJ}w;sDHXSd%J!RWVk;ewyU?_w zF>e<&Z@?^lPB-kF<{J2R0=5_mE1HwKblSpYl>7T-qkD*c{*7>0QNzHgwgfyVy!s(&X5VR5?1mgd8H^yq`jf8f&HG72eCs|U?JGZPq&FPH2Wx^2kr}0fjlq^M0_XhA^~Jl+ zT58(RemsSBA}S0vbi(SMt9BwRwykG~_3HfKTu+DG;8MOpSa5y)alp#z^)cYizLRX@ z84M!tB09XK6_dBcY%JUbR)sXw57zFKXU*H=`SPsfgc$rgw4eq7c6)2CkSpn7+|PxpE5b+4<97s5??a;T8!6DBpq`Gl{K zUpzfJedI2Z4YiIL%wK#WZ(T*8G#1pjAvCjBD(HhNIMIe+{KlHpXXE5+)p{-HFu#V!CYS}ay}9{=39dlIL@`^L<}Iks8JqW)=Wh^ z{fPwS7Eh>KOb*p_rb~3-MvqF+C5nvu01fKluNTgXZmVo>1Q6hV;6NQ^b}pkNQ@;D` zB({czR3PhApiZV~R>jHjjwG1lvpkkQgAUWIrZkln(VunPDRvF!VyE*Rvhzq4iA@w3 zqH*M(pX=E6;RsWdCt6ma8~-)ZqxZZzaflfCY)WK1#lIK-4L)eb!1$|Ej2$Gct`x)> zgsH7f#|TCF2G^2}$q|3tQQ z?H0$@bQ61))5q@U5}UoGyvErxF%B!LerV!0q>x7>&_Q}OX4+W-SmShxMU0#SUWwx5 zc1ZEk@+6mzYA`zs0&!My+;z72#r;2cUMGoFHR`oKB`#T`4r>dL8L!*=jjRv+sE<|) z%sP`$=GpVXM6}BB6cNrdfJ{i{suf<_rkQumBl9G+)YGntLb@+_A}F(x?OY$o8IE#d zreD!CyyN>4x6|Jj=*i9*s1c2C#@Ue^=#j*ZmSBODQ~*j96o-7aRx(X$2;8d$71-`3 zZ=?ntHksyx0gh38cY<*XuMf!Ow$@#nZRY2=GjBJeaX{1XC|~a3aedF7A32!0OmV*R zbz+&$D6j9B8l(2E7m^1qK!f!-4iMMT?Zpc-R7&ozi=DU)-1XAkDxO^z8{6(Z&?y~B z+hm=jTeG(!p&=C191P?%=E4>iR>7yBJt2VnBqd~ONIBEgcGs>iuZ|s4rq$^Tf8d}F9H<>y zOe!PM8%`s-lW9^*GYRWn^|Vhm!`BWuUY2P4`bUJ5*e?u9o65KiJ0J`E#MFc;7B?dRyWh^Dhh?q~C*NepK!5 z(wV}WY`==354WHjYE$)n60pO&NnmhRZ<7ccA+VbX5cbzZ5YJ}Ad{76sq0?m0xWZ`u z^X|vnvImjrZ6B=;3?NCi$Irp*NYS$k&Zs{MPfO}`6EPXXw|jt1c%PT=U9efPs~AcN zcdba$BNywZVFTon{SZ0r8)qV7@|H6|0>-9%%7CEfys660IJg(tZw;~*Iu*T zo**Kja(V;O4Z9G&S;-`Ndpml>Z5 zj9Mc)S{7{2DlbnAFZ0~re9-2~OmV`VvQIqV4>Ld>xZP6nVMzR>2=^iR(m78L3BpN4 zk{zO!>9A8iWPvA~Colx7AWF-bMwEk%ILS=%f0*Mj7D3Tr*NJlsIuP%QJY)yaL81Ps znFW4t4-x7a8u0KWCEG0VZ+I#D))>83{l~|_hfOQi@Ay0%(&O58&R>-MFN2G#R67qv zK^^81_rBoDzYuva(S8LZ0A>^XmmL^svbOWIhoUDA;)cs9{}E>0E<*8VyonCR;xDh; zy2AeQ<}+~Udaaf}(P3t-I4Xr^uYfq9t_rk;EL_-*=c8 zn)~6kh1IWW?Zj=V4F>+^I6Hi{AtJcY9q!dvu-%k~@;&UCDoZ@#G{R(BT-wlWws(qg zdVYZ7kk^eq<4`to5g6pbe!{ED!%43Jh;0*0q`ZTb0hkIrQek%HH%G02gMkrrC}|EwM*d(7{H&(KS@O8mQiFV_D+ znhX)WA6kn%aTvdCs3!KfCv3>gH0O6y5LkQV4zbVJyRFM4jN*R^q4K9On2v9E}3id9L0sW#3G3V%WU! zkIm*a81BHfb}CNi?}}-|9HCPX9e-qKx{Z+bKRzp3eXe>YbY9aTg)ist=D$L`WIm+r z;Q{gQZUAk3{_T*1{nY=0c@dBYRu|6d+L|)`u0*PDcCHmEVE;om)k8a(C|HryUiA8!{F*tW;;m3eI3;SJ3KtSD-5_k#caFx{_R(LqO0LF}i* z-~-fo3eE5z3u(4^xF(5XRmHJJs~2NxR9TUrq0JQtY#@NzZSPAZby+!FG(-fs(N2mls~0N#IC@na0z|R&9K=$t!D4 z-FUaK^(mb2KJDwvZMd@3+9tFi-e-K$dtAz}w6u;Amat{<*s(JU97L%#Wj5XdqW3tO zj;~>A${a;nOC?Zby3`FoQo0}AZSIf*N`mWEc!%MQz^dH$rTx5$bB|PvEQ~#SalHl5fk2@9kVD;rN^=`5 z+7qc0nX+&RiTRS*bN41&4uY~tCdmPoF$m+K-k4b|^yt~)Q_4G`TwS1N1p1slb+!D%b|qMBh;+^E;+OsUmLG|&=^Zgt1S@$_HT?ty2O)##`$|)$4^h3{ zc$o?s0IsBR2v1%C~%!`1iYr;{Jur{i7j&SV?tbcoj zTz`f0vTZ{x+OAYn^y5{Xd^JtQVbA`z>;s|nlG1O{<`Q2|&~finx2Z_i7m?r5qL{o_ zda7p#!EXi_chdXQ%syg7L?MSBxzD!XfgL|M%$)IjmPQ%oKmwEWi2lSAeB?>;@wZ#kC;)Tmc zyD`=b4{tX0;4C|Fi=1E7onBG@zy5}l2omJ49eV`&??=YKHrwVL_vNph+#Pe=m$Q2R WR*0T^fAQcSd+I81lp=bZQ5@7~Ye``P=1ms@8H zwD{NWSife?8vfJTCymyuSb6u}T78Vxf zk2|`$x+a1StWj|XF0I}ST%AB3kb&L2aN)ul6%UKJw$9E@3i1GODrOXN;W6iNVDL4{ z?iO(!z{nKjA>h*H(*eNaz$s@O6%XJTH~Ed= zp9Vh6ib^ZaEvqSc+X)QS_bQiyJSg#8FATWhct>Lq5?*Q~K=Ieqf@IX_EE?}lJ;3!gswRMPHC6AykH)os## zmL*1T>j9sX5qwC$Lkzp{zCc?0QP49g+x*2mifyu{gZtKn;ymA{9kX#Z*Mi<+io=A% zil^=g3bHkHq-0vFtJyR{l+lG$_S6)ipA`)pKI>-{>ES{-NAV$2DWrP77^4e0|8s|( z4_T01SM-*Vv^Es0bPX%3Bxe=L!Af&1Xg84=R<$y@ z$vGrYkYF=qM?TUVN#xI6C}1vCR~v_7xX|)Gvg+{Dn*J$09=%;raW7y?%gMg-lSpn< zrz>6%f*LMtD~u1J+N9jd%jTJsks}zw$n~M#i*1dd9+~v=OOixa6}54!U@rqC-Jm|5 zE$b~VNetXO^sGDxhU`TlXEWZ{s9MiEbVt-p#xdZ=IrG)}6(0Gl*`a6(y_3H?=>d7Y z;2<)WZ^54tsryWBzQ)<5^af|fVx6eKYJ@9UR^my5I7yZ=Quk7Vc#WO+VKlk))xwe` zc&a9AMhE<^Bh}&!--1|e=E?06rEV}Zcq(OM(X0e<|Au-#h0_MO`Fr3g)d_u^k0=s` zC?fDThm|`AFAUf*Xphqbi^i7((DkBka?$>qUHl?^#;QwuqAjBZk+Rg8$8}rK9TX;w z$jIn1vz3qheNZ47oZny{XB2A5M|nIX6-NXqI#fI_MP%YFF!bpczXw@~V7`4?ro zX#s`Vvz|e;0p4!%Q}rl8#*9(wsf3r;1{|#Fk5V(hZdCm(=r{Q0rA|`;Wmr#e$7p_H z{)^}I)TMI0#)cNbe&3j^x{lvHTo+=t3XH&VRb8wu_?EnRId7#_=p8Yx;B(z(dv%&} zkU@kXq}DJOIhTVfK38$VPVQX5I%oEs&TcPZMXvtp16&Q2q|!4vO|*4?gvyl`p4A$d zt+o}l`{VJ90IlOVb9&ULu@kFRQovkzss?3vGZynL4GAW`G?Hz}v*oE2)q@rhM%8I| zEtzLrZKH}jO!nsnwG`$D5#4z57fM;v8YRrC&e@mQ3WvBdr!PAWNLp_nW@tPQ+C?Hl z>KlKRAWm0T1fgX2%RtKJL#hH8MopbV(E%YI!yZZGX-`JD+f>Clk&EIAP*M^B6PnhT zwzO7i*W-%Oa(yCUtH3d=BAL-B_X_DXBputAC1YI{yU4Accme{Zf$Yy!fKS z_kjIw>e|Xs0Mf@zZ`FC+eQx^1!`l^sY@_8kvj<}9`7#e$M^6N1-lj^q$_3HO<9LFDAe$H~7`cNP$sFQi2qmLUzhk!$PI z^q$lYY5Cc4hd1*)%++NY6lKaqcSq;8VRG$D5P24V_?`0LuHcCN6t*Y%Zdh)$;aeo! zV%g4D<5uP$rjwi7GYv$MW%Jum)>B}7%Aw)z%6CUSkHX&HjOgRP8bM2=?VG*Af7qcL z6y�ckvA5k3rAgySIEc#mNVeW~cl`#ReW$KN67LGEPbD)!tW6m{+VD4`5@+;L?nL2dmszKcae(j3S5*BLS(3^1n)x( zU4FkXz*q*Ilp1tY3%@np8Xz*se!DQ+SG~znc_LyzV|xc1=9X>a?naoJLYhq1@32$R zDOqe+qt-=?&*Tib;FeM^QBOguSfB!}yrkQFZ}dw}AUfM4qHp#+9UyY>z5&2Hk13xN zA^4ouVBaZx{PtW#wXhHMAZ$m@8`wj#_^iio+>?U2)6Dwp!+qzb98`QS)Cu;_lR?rv zw+wx#tzx+Yq*rcW_uOmZj=T z8(rPy(KE1G1#0;xpL5ev6&vdY`E8k#E$?)z2e#)Ij&0=_w%Uo}YIGz}7+JncfcgYF z(PvwH1QuDcuO0F*9aA`WWvCDaQ58tM9BpRdv#t9w$2S+ylXt;+@$Gv zq$ZZ(Cu7>o8Vbq0W^a)9q39O~6tz%cDrs%rJy|7lF=0M;oB^I9O}XPt1uRY(@9*)x zqlS;zi-16Qj{y2fauT@{wtxP&h6etKym0VyY!L>wcE}h~7m+u=#d*PlT0!3NxNK>< zWdL-<{`JSSA|Pp9vTOF8`Af|El0oc7>Xf-@VQ=8Z=$rdr%61(*Q0G)at{271 zONccrOHthqAmav#)pUGwd)B68f%s69iz+=~%&+R(5s^Eo>lO<&lvPrL}r(?C3o(drD8%^Yam z=N$m;3pQ1^REiJz@DFTN4}hc@m8|0a>ekYWBTUe+D7l%ZNZB?`!g{ed;BYQWgciwX zhW3{_+DsYYvZ^Q4s{{qHUX%$$H|OZ3E)A7YhUyDhivzr~LHp6>)9L)7Y5GjQ*MiNY z#Ie54!(g}N(+cTmZKnS*v5^2$9?5n*?RBg4?D`w-rR4;Z(R@xtw8?R1l` zNk6UCmihiI{Vh3AADJy1VOX{oy;1$>bMNR4ka^Vo_aaYw5|;;__6ch#YBNl!=bKNI zO_yatj0G_}qU-!Hc9{FQIAb>Uul4iz5H`2#Hbb>?(WSQH7qE;ctb^6WwamnOjBdj+5$lUv*`X!E!aR5aaDEr>(3^1y7KS zuUQboqae7>mtmw%#eRj9TF9>&j9ge{MUXQl!qTVOVfl2G$84V7?W1MED`Cj=^h1Uz zr61BjyF8oT6Y$)$uzd>GZ`#KQ^)7MbXFqmk$X%$MO4>8?l&OVLXB~)oB)pSSS#i(f z&T(XtD-XHyWM4&4Pwz=7Dn4hlw6oKSFO%I#uNM3x(o~=hGAHB^jZ1NcA9Ds4-qW$l0C@r%tP~h6V;K44A|9JJC1c z0e)M@BQzfo?p&L0U=G+KF$23P=?xK~QX^ndgazfpoX>qB4-J1zQ_DHh)|(h+)G1(Kf4F~GIc(_yj?5asubmclEL zL$8P<{ivpt#nE8y9B8II7Xh7v>Qkf2CP&H6Hy|n-Ql8ub-(y(?G0X!C5BD14>L|u> zAz*CrM20=}J)H z3f7U*WYRE|_#2f>I!2v2=iAR3H&kch5v2A?#(66i5@{RA%J zR0lKY+2v!1%63zy#(unz2D=}{D4RxCc7#<94)yWb4xSkSyP2S8RAWNMr%xdLNAXfA z&np+`5JDw;nqbuy%vvS{JF?j{G}{ijh;9g5Vq3A7)RBx?;6J(X4a+2}j`)K5<7&Qx4#D??oi+b8Rn1d<;Z<72f8I))eyVB2Z zpW?gpP(*uqax zuHY(yC-=Hh6t%P>t9ea?kdaQbvOHJtbaoFalg;Q=!jNU;i}}ogrQNpKF~DvjVLsg9 zwsMlhb1E|!+4-AsyJAtdqci+$L@ zDw4Rrv5kbavi3vpfs{EW|AXghYs^N^cVn!UGgLP(WG=}|Sa>^ghYwAT(3-L>dxa6a z;Ct83DRLR>^!F>=LYedSHq+W_<${fQJLFI)UGu{FH8We4xq@5Ou5&HUB6griOcaK{ z+5;MJC-{_mV0Jj3Q^r!M(E+Z{?qnaT>aNr*F|%Yj`~9AaHPr8M!R)9`HY#|^-6D&e;*D*SRf`UW()6WLydnW7 zp^pvN6e4A#EpI->WFQ(g#Tb>ssNlS=oQ5!WZV%-=gPJI#($~upL{BWA*`z3@tQL6i zilQDa+9mKU2<8GAS0_Z%FX%`mCb6L?14bdw^1cjAw{u)FUG){sQ6l{(cA#% z#m&>a6Sx(K?ci;G0`5AVkH7~Ka5&H3jbmQ|fgQ9iQ6s)39th!*2c|>@qdrIjG5*Sx zk^hDv_cGB|PmiQDeVi+WJ+(eX%3F4IzP)xtMIB=@Y*pg3^*GO{)V7jU0h6MxjO^7< z_gr0~mK0C(*;u2I)fI&stxHV$Y|xD=T5z3&k1EjF?ibz?r4+ro;;2JYBL5dmxQg5E znh_7Ur`t{}FChQN^-fcc^1K^7Hy+o zsZG5KXk}GvJ5ww}rtVOT|6uqh!JQI}F ze7!Ardr+JWw`;f_E;!yb80nL=UYgwI_rDAXfwW2yt=O%* zKSO_D%`eg|1;27ouE)Eg>hArItRqv`zU4TylERABEb~EOM`W})Zqiz{q5}HuJ!Z-9 z;~aO05m?befvI%WF8vMnIk5L~TR{T$65m$%g?k)-O1-v%H_8^>Jnh!2!yLyKUBy_g z*;UPtB8?GOp!NTONG%LdP|wz8T1%;AcTRoE$P6{LLJG_KrIzN@xS)pK^!7~)hc;r{ zi^^rRh-EfkT7;8fDx!pjK*K)PWi~Oc5nrkh@P{}iz!eePj{@tw1yX&#vW$PvI#{$& z^=kT|_}B#Seu;s{)gWiVW!3UF$rJT^f8#D9+^AJIfPOJZsUhHZhhst_C=%AQ=Uf-D z2LYz|{aYA@j7QT@J)FDrDKJl4Fsme^W7 zOx2e+^&xmtso`Jh@PDgkICHD1B8f&9Hh!V!t9l7Qoq>-+b-7lup5R+Aed&h3kwHB{ zS(R~)%w^hPgtqQi`bh`pu`Dx|@ZD+@cI}4(&{t?t;;K|rS8-{e;7kl~DPKbMO&i|= zoH0Wi4-E~anfBd#boEO$8%q*HX?Qu>>3jNW0>wwZ8JSmoIHFpb|I6?rp@*YIz_i0& ze`!WO@YFbcTq$_HTgjKf>5vUK%8ymv>w2CMTm+T*>6_68r#WvH8a&n36`%Y{1bMD7 zh?+|gp?UEhapo(n&48p)un~=`8@>|c4M+s?%Wz@APhXC8OCR@4ZTFj+uRrHA#O*?F zvE2NX#S%jd(Um9}In%|Usg|#1CWC^yQQ7k-$T&6hbl9se^^gvpnvE7Tq)k^gTg?T0 zg#-&0-t_0@#x}lUW`55`Y4!&yFHGc%nJpAd1*UuSfazmH7L}g~h4N2+iqCUfn;vSY zKmoHwd&h-9_!P?b3&Pn!j^7HqeNDL8-XwGb}qs~^p zBSbGfW7YXJ+ieRMYALh6ru->^VQ+YnfECa2Er`0*gMxA}%k>F^?S4-+$3A>Va&R&K z!tU~=#G>(leoQZ`r{azfxxp*Y+3m9=iE+yj87z6M#HC(7>W-uMF2G~xILI}`VLe3$2KqXXg%zjqBr1uTY(DhYmj|A)ucS zLZHt3UZoXK4=i*H-B_A=$4+n!`NkGK2Z1sdu=u!d;Gj(sqZf#w^RL$VeoDjv8%FAP zCdQ?M-Dosr=7lVw-eKF-3Kn4it^4ksQ`OOO>Zpj3tN5X03;bC#G?$oMvTxP*?+ukx z*|J>GycWaWoAX?qRW8de;$;JDQuSLMj9dnAyNGIDn??=$W+5w}@(q$zKw5pP5stDe z`#r11lx@FbS|G-Z1*!_gS3ti9lZ24|@e<-(Z~iwn;mG6M{6@Zfr_5U6o=*g4OeTJR zsGn=90WKmyLIJ(OQtb0_+4Do%(cvqYYU7L9`uR$X01725x}t9m`{6tVgu4~?V}4b$ z)N6HEJ)9w0OsrTCHdR}&nX^_P9<8vi7oae&u~qMT975|7Ok5}BXM17&KOJTXd_a9v z%O)`qMZ61_;(!@&I6f z#ck0P4k48S80f0i^bK$&OZIh<9{==?#$>()Bet0xc>|+q7fHt{^5&U|y(yjF7%4U+ zU1LHyEQ%xQiTv*h;+E6tyH(2cLfDb(m$am6zN0W$K2_RiMIkleM&A|oQonif%3^`5 zo0WAqJoz6$JG&%j`LsYok!$2O(tdSCdwsrEnU>T@{n>7yv+7w>@lw0X7tsB;Q|%?e z6m^uNz^eiS>*jyj`Hg*2;`eL8Lp`{gwg5NS0b=b0 z89>P#xdHWjZ{<-QIU*^dz`J%+j{X4AztV7l@v>>;*0C)Bg#T(z%4Wc_xxWM2U0Eed zlGy6g%DRZ@Z@v($RqeTGim*6DPtdFfmag%oZn~uaceUIE=mLmZF(UlN&N5zv^=8wBM+gh0B%3~ zIHcXUgfw3ZT85wIVE9`|z|(+m5}*k{K+!b3g&KCy-N6!^7hY+@{V48QN2SFtKd_`Z z22cUSc7cNW)JTVS{)luw4wu1hSUWmgKz950QTP=iy7YjC*J_Cn0QH+TTss0P2LYG? zBJx)Vz#1N7V%%tTXeh1Ur#iQS@+K){L-h}-c^!tU_3S^xGT<55U4Zlch4*|V_doa7 zFHO8xlDHB>0BwIAg$#h`Ja-#F{a?D{ zUJ6~fy|?H)$p4r?UQnf5eqPNowtIbk2cqgqa{reHK5~=yxX9r6n{VR$4vl<`AkEDd z$pG%Yfc?q^QI0sh){#FV&WfDC*T-P_!qP^cxxmI6f$1%0og)EGRbc!%x;YoThe7fC zeXf0;6z)Fl+*z^fwhSk;`f$2jrYC`n4y3Y`3*j7Rbj|xbc^=%X{2I7tg-@;cnU#pv zivRtMXE@C1V+yT3Gx4*P7AI>$%|6eX+ir^v0=Z@v!01-uEr-5+ZlWJpI_J@k8J{CG zK9ZB<`9F{7-yj3f+CS1ZM}SoeuKPSwe`7%)oc<&H{HTfnvwUP3D*`BSh(%e|=fncg z%3cz$f=J52&#>}CGjX^JFlt!TrB(a?kP=oH+BN_Lw`E#5HvKXCa7dXg;8s#a+MLMD zLC{A!vx;UW@WzF5N+=OyaZyB0ddIgb1gIr&r>0K+YC`&2`J*Uq*PC#Fm8hz;H4NRb zv;77ke1D$r{;}Fov7rOI}lXJK$;BmWBI#=Vu*S7w{1<(aw@moH>&7Uz6 zz|enXy1I+{xZsN26JLXPg|%<2*(z1@UH_^EUO!xha^Y7Ba6{c!e^YG=cK}i(4|c0R zyp!#E`WvVF3!y`SK0xDTZpW2!{W;>|fp0?AU!nY99PnEC&>7B4=SO${wz7dvG@g*k zTJXeniY*KOE6|Opa>03R{@aLONf>|AQn|J}!Bg|$;t6~HwwChm@x4iZscnA?2*;ll zEI{P@E63s_s((ME5ww!A|E-9BD}vJ|`NZM=7b7BfRM7Ic$gA(9^S|b>Hdpr2Z+T&L7YD!IyBV88RS8hZZYRH zS^p`v{{x}68{kaJb{2OPm`V(`nI|tB?E1*qKlK-X*dzz5`HH2WToAF2QVo?&X+ZZj z?6>dG+0S{`e=HfLv;yXW?sjT*BTE}qXn&6*e&$gB0nTFHgR;aL_UPkA!~vRORRi#i z#a-sfoYI=pn*L;_&m8-o189Yo9_EURR-#*?L)5XijqM|6G+%tD{`etKtq@Ho?xHc^ zHGM$f)VoU5mj9gY1Z-EjdRfg^hY#=MUv0L1@}!kMH>aZeJVLHi9-n+1NTsu}ArXZ3 z1HhaAG!8I=BBj4ml>L~ia^PwNp%dB$=FO<<>JrzZDk;;bk8eZ%M8Hbt9PpGcL(?}Q zBc>5}i)grZ+{cO`IJ}$`f*APEOXs&8@Wy(fx8JFHeo$MATpRu~_HubLqh|{^a zDtG^T_3LL8`V+{%)NtqmQ2KA>*!O4Fwr4k0@|7eRc=1aA zb#O_FLSOJ*>7#tv#rXj*_)4DdqU?tv)K@zG?$ke$Wxm4Ack|qLrTmI`eB}$@o%&xe z*mtM?w}L()`riurI+d?zYt}t2Nv>MUc@dXZUVv|MA4u=*I18N~)Qqo9F6R136o9CChzRq#vqOS2DMJ z<;oX=)l`ZU6}EH8{c0NjF0uNq{9m&5m5G(QfM0BJWSFtLDhlu`#oKBs{Z9s-I0~o- z_@$=cnG7~cWQ^@A@~Ip09U(rxIr&8LzN*tXD-^4aQSdSZPeq_~q*6?mH(eXV1xXT1 zu|?E0bPNqccjXPJ6z?6lKQnxhs5Z|7Tm}|4-h5reYJk6-aK+JIC8|xnp}O@Ce6Os0hzs?`3I|Q36MMu)%BuH}-j?A!=+zs6y#uZgvV%7tz|`%T zbUW=oBbyh`JW$8C5c~Ok8V0z-Jp=UOg=2EYYz>NvztEdK7l{jz05hSv(jGM!#|J*C z3lL#%Y!bet<;bK}M!O1~V0T%&owg0ez7vZqCV^n>No^bQ!G2BA>8!q;qEAJf$iV@q z4pe6p57uFP3s`82`Y%jU=Il8(mUO>1^XzCeG{HUDUAW-xdz6Cf?IEptKC}5;d?jpvOS^C}*P;sSR~rq- z?hh^*EA~_I+lNr~=1p`@PK__7mz|VAmme0GGZM;|vdZj~Lnzs*ZzK>)#B7%e=Q(S8 zF<`AgB5olv(MKa!g!%|P zMfB2VABET_mY3GxDZd*>c-3CHK=V}mMMxqMyAUaWsme!(DNim$K2qjtaG^KaG7pI| zwIR^IM}#$ zqg!9BQv6j(x0y&Xwx}-PZg-hXkvZjVH8@o+J?LnMx3)-A`B?uoNl1Dh2^p}x=!B}f zi^~Ih%G{p2Ww{j0izuQT;x;w3+T|YD&3>swnryh`Q36|JVV%l)(dWH;_xq;b%zjAQjRt)j*H&z}wefaWpKs%%A48t&lQuxbT%!|AHjCe7gLocM`85u}!?a zx%3XzqwnUP_QJ{jUsY^o?vooo96CtGJF#^=+#(kAjU(pHRqD{(Yg3Wt_3Inra#+2# z7{+oES?YV%1cD07)Mg*sTwp_q6S_{{ z+SJDA#1@~6cw^Z%;iGtTNNgZYFS}r@`|(iY13gtE;a`Sq@9x-6lV7^g`ijtx^geC3 zM-Rtm@$?oov{(K)v`35Mgch@2JNXEJ)_v&O|%M?p}FF+f-C5Ud%&K1pXhQ%3}qCo(Y59x`e15t_c-ls3zk+5cVk6u2)<9I(^>(b60C%S*41TObs zYB2h$@A@X4EOP%yvf;K|)XuexNbT!Ik+c=L=5y8yfJTEZbx{6H{6rQe2D;`}+I|!RrpYai@u;qdE*JJ74&edZp!8cu@Eh zR1(aW#)b4>BI*@|^;Lf{L}0pb9j^V)B>IOE%ah&b;lgbX@CHx9Rjal$p1y--mBx%k z5FzRkmt_pWnb&5wLIRxeJy=p%zY`f^R4o1`O$oF{mLe#d>(h-X=&iNB6ze$!0YWK zqmQvulDK`47x*gX?S%695Z+5bF7R8)K8ZU~PvS$8I^Q<%`TgpX3|tF`z`MXxdQ_h$ zu$DbD!wCq*m8U1&0((|+r}f`Iw;2R2Cn+wBNy_$gmx=C;_PV9wKT_7r#2ryS;KKb# z&_F6hkpZslx&jAJF+a@hnGqgdTur5JXvP(HA8Rsb_yoR>@-nEhHBz_qSYr9CjJ^6W zNTpiTH9{F?lvut&`tP6jxN#R*G0lQiYZNP=4aP^jrMOr8IT-8+SE!Xt@bWyf{3zEr z;tg_Eh&&8(_Bbu{ktBpk5T_&s`WBq@>m;p(6A-M}0)ioKDf2zeA5ow3PA)V|^f}WA zk?=e~YA&vgTkFCsfjoZjc7TU}>*iL$bIl!7HU7>G@D1_H4ShViJ6J`w%Et&Y3|@6O zkdG9NoftBUl>O)AV07NSmKe-!8ms@9qD%#3E^c_d%PIKez>;o8w)0C5kf+U6UAl&9 zWOR;{JQthv99xfQ2pW&RES3-gMm-&D1l){gzsoQ9O3aMNjw?@jRNEWhvGnvA7Ket= zE-}9WlbQHZGtX2?K89jlX0BN3kQVT_>qqxy-eT4_ztzgXclyNB=VRn)ztTVkY=Wb4 zM8UorhAz7cmdowcQ8F&|k1F z#RqCM88+ne4&e}vl$Vq7-b04?5T%Imhk-A1#7oOl`19Sl1>BFaYBCx#P$}JUz7R{- zc-5ka_c`&dGbPeqsx4OKxv4DCb|6r}4_gIapV;%_`CF~-iiu!|W2kaaD_{N|_oT{= zcLJUW>*Ho9a6~9xHAxO%$i7rsAQzs|5LtX+TGzYdt&{vXbW&Y(Q@`Ml*gJtrdjG*_ zAKhnqrp$FC)30_5O@sCd|d3c3*9$xQdUgRZJvY=8EtnT@1 zOTA0h+~)(W7#`PhEj#+v#X6n};@i`M^t{&GY}$iv@LR7Te6UD0qiCy3*t_zdMBvm3 z0pHE@eO@{ar-SxexEy`Z2kbBrsO|4Q)*p&oTb)}j-Cw2XUTr0yw#VxUQ+$W z=fwnRhR2s|GcrGmBZ=4 z;QMCm3gCAY8BeqC+M<1ywS^3DOZ<0vi@NQN4FZI1P}~C3&kZ1V%Zm7J!Vvq|1%&%M zpXOdj$S2U;bZmJ6H_U)qlJ64^BGkA-5n|BCAvMNvzT+O7d*!kv3{&4k`Y2nPz5q7f z5{%aaEB+F}lk3P1P;iIFhIpT)2~XLI>R}Q?8cP$g@_q;pVYl-985FR4L#ds)F?4ZS z{GtL^!$`7n)sol+^~dtBCO|zk%A`ihMejI=w?-MKrW({30f z);g+Qlk?W69^wjhGBF=`U9I^jx$Z@VZ(PWbq4?$p%|t*HB`!Qk9?3L~aBX|gXHb+7 zQs$Rd?jI!bE=U35IW?K4*_1K`o0WvVE8`wFa&_K!S-}M%&*kJ^cKYmCT-~s( zX84f_&&18Gr+bhLb(^I~FF|(&%M#T$H-ztOC<>-vAv2lXb zqg;ZCyAy)40LXZ8t4*(U7EE^O4qJNL#aA4bX`)Z=gf|Q0%WizArZrP|AIR6{u_ePG z4w$u){pTIIwRZR}PabTH)5p1QDr+CTt$KV|yE=U2?5Tv;IF=7D}eq?f?m**}fCL}wk z=ZedY_S}{^Z%qR+WmG4x%EKexm-SRzCS^^^6pbsd6;O9QfaLR2LIc1r<8!9(K@&s9 zOQ&zf!5O|{{ql>)c%hI!IJW@9#>7;C>C-YTUL)}<`?&T~yj)Fj+5I;qBB5@o=*Bke z+3NzytWuuw2d`S~jSeqr7nbgguHBEH)CcRS-Aim7c=#gu1shM(e}F%_s2y}s%=?W` zdP3FPz{}*w?(=657CKlRjCi5f)lQT{ovSRQ0f1v(9{i`X0&f~Sj068H2`j+G?q4!>3I_n`lyKK-+J6XYP30c3|FZKKB`&tZx-XJ??+D5|UNi>C5KKd<22!=;5ON^8^$jv(KG9?}+JdW7Nw9V>>`?asLbsNwGrKxMD5H<7CB z&1-8LC5W~c^(<@|7CRoWTg(CYd8;qR<&TRP}dnqveTesF7o2s{0j-{4e~;y zYP%Zo8@EAdjq?^EpI1JZN$%^7 zn-9|%@dyq?oPWUPwgt1F6&_A5Ip}}x+!b+t<%*2DV$yvdup6v#IMZWTyR4-*h3Q=( zTY|k!XJ|@-;l(p|(5mK5R z+>>=}}+FVJmjEWeZ3kU022|b{c{8%0+x6xq1%dh2R}32k z7smeh;VdRDMLv2}XK-M%67(Vax(&4Kv>S$&B^@13XRwb8#s=iA7q9z0FH9r`*Z>4i`R670k5k>6h6IAlfR zgrqwe+uC{m=D~vb2{bF=SC8H5T;hgqyx~XG1k6|ilIOVl(+3$g3r5%xknO6vEM(v(oiPI@>GE3cIC zi=sDRszRFW>Syk|?^Vl0LD*y@xyjs|?k{bO5h9^y7MYI{IhlW{d67%Kh zXd{N4<`FLcwiiL~=ULz+I>fb5mvXw_q>MGXXobAuRT4C;K%VR$hG1S4{+jqk?$h-L8X0CL5Y}3*&5h=kQVQSAsGY`YM6Y5W= zilqZF(~4Gj@oaO~l=jH(r_3`I=V{vQt~ZPm@*y{MDZ~1fZNog%zZ&&+(0-znDp6@~(C&=7C-FQp_31)w6+lV;y+Mw#%Usz_x`(rND3N8tXK0c4#Yr{kI$@ zOudki>mX?|gWD7#o;{9?*D2$rCWu&IPu}!9>2-8!E4_pnIGW8@-0g~Sl7(!E4~eiq zuGbQzEhPu+L)WHW?2gh6sH>x7U{m%Lwomd0dT`5H7~syW`~7nD?bCj*r};WhjWp!v zEB+3IYOYWSoqiAL=a*yaY0@{Ajves~^jF}D5U7j-9&4%WhUX$I>1@Rh3m}!Fu?B{? z;KPo&YOZ(}Gk}kTxw7w4J6$_~ZMuwyL9l0)UGUuTTJ$6+x1ljlNxLI1;96FBihdHO z1Mt%Bw;U_6n^3A+dCyT~N8PAT?^*T%$J|F}kM@npVo2RcLV#gl88S%LHk*a^KTn+$ zz?cz2HxIdVE^QY9_CF!i&0L{?vnhlZYiQvJwY>)Oz#escJjV)}8rkavrX_6))(k8JHiEwVL~WuZ94~K6INLop)M6XR zP?$>e|Jg0(I_1>BesG>1iN3g+bVF;#SiswvU8zo~s4<>Db2W)*WEoraOw7&~QUfyKnS@V0euvAr~{M+CaeJ{N=+a zZWK3}ZYhe(b|}{&w0 z@eM!;1*Hn5DMy3dZpEM;_MOqX?#gP5^QL^bfAd~kh=xjlwC{0Rg$uVB%5Twr;cVQ{ zikj|bc;L&B1!8NYg@A1f4Jy-&%ph6&>FN=Uba!9zxj9glZwlcL_4p92+pQNH_<(I! zDsN+=2A-CoC@eQDfAXfwlEiJ)DCjtHe5G85BLet2LOY&wu_9|>-ossaI;09^DO!nz zvWtG=1t{DtV85^UB1|mbXxOI_RS>iXRk2%g4R9CBjr&y81RT_1klEUEHp^QmCdlR4 zA~pY(n`h%fT;!c}9twxVc~oZ$>`xl5e&Y1N%@1Hu)VrlSRlgKcWh49SjmfHT@jv?3 zi42j`CG312VWJ+lCK?=o^s&;Z(>rRd-doz8d&qcZ7hWWoeU#FdoqFzZ7)^%4H#BXZ z@6_^PA_KXQq58!C5;s}a$A5MJq$2$fEbi+w`f&H=No2RzMXH12^)5bzT76-0JHII9 zEYj5vO;w?Ske-|PgLf*QVe6jwRfF#Dp0!ICq7niiz_n z_lbm1hGqf~ZH0t<8n3m0B+TcO^D+4z3YQ?1C2`VDBo@6 z#H3bI6FLMs_qnppUkA2fy_i$R=NCJlRIjw1MGUtT#)&B$8 Cm-oK_ diff --git a/arrow-data-source/docs/image/decision_support_bench2_result_in_total.png b/arrow-data-source/docs/image/decision_support_bench2_result_in_total.png deleted file mode 100644 index 88db8f768694d43b2525700a7c75b59cd9fea25e..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 11741 zcmeHtcU)6zv#xFnHpB`dLM#Y2h@gV>Cen!tinO2tq8lW%5Fii~w*nSWI?|+x8X%xZ z2oaVD3K%JZG%2A4NFoFhLP$Tsz4r(D{c*~@=iGbGx%vH4*1KlSJMYZ%%*?x1qR&{H zh_BnZZrQSB;%26&&MjN^8&UK=zh;GKL`8VrRP?pn|D1{8vf}1lWYNJ&4+ATMWy?xq z*Yhr~5*-(xv9vW7eT!ZfFJ2UVe*5-qnXaFqp`pFKy;E9yS67##qoaLlXHQShGHqW2 z0|U{xNbJvX2M32`+J5@_`rX~#qTiyy1tnU(q7jjvg|j~tE?n|MOmryvEm9@=T$o7P zM^7iowxfyluL2N=+R`#>!7!0NgsI6;k=_U>hlW6pWP0@3r%h&pt+WRkCCbfCt zz5J5`LvPu#Bf(~;3~Ym-lNsVjnR>sg(h6#7eO0IZ_D5#JN*-5T{Pa13@B`4#Q0^fRX{J;OSK8hF+%NZ%ggYAIF z)Ja!tczt3?tI2CuW<`s_BO65ire-n2<;)36sd5NJ zm)Cq?CW(bF##cEM;mUV+a=neV7U$<4q6g5fUASOp$NX5|uf)=o=g%2Hjf1>0_h}Po zRxEwbvre#lix^&#=Fr$5#Ij0nLcfG5;QZ(d!*?IJx#{%XtWJYjdlW6{sCuyK{V==J zBL%4Tohf|WF)M^+wn+43Xw);axd}>KJ8pbO_uG z6m`-Knx*C5-!-1EO5N~{ehV2wnLj@OqX-4flv@;@8_P&Ite4R5{X(k^$X@&tuirJvizuqGxc)o6rj^Jk%K$5PO4P z)*XP*tl_^iuxF1}RQH;p^_pdKN2~2*g14|#$sH!4gC~b34t}kOgkq@@uL#8eTFy{! zS2Yqwds9-RJ#jQqSF=hZme@tYVYp!}b#mKQ1e z|FqoWtUXkL40N~_xZZth>Yi@*1%|nRMmN=F^jRbvsW&O$Cv9{g>^PN7*!LXUOW)Gy zZ2W-gX+??D3NGxWq7(c4cATF&TwDFNFqRWV4GR;~4PF;AVjkEo)-kd!SgHYUQ*3HR zi8|ooh50@OBbAblzB0916`4+VOwBY88^&s9lY336kER_}F!JfqI!B)QIaX3a(97Gz zKo{)LNGq)#7O2w13LrHB%aX>i0Xa@n=-4(Jy6TO`kkuT@@MTux<&-H12*h=c+#+2% zdJbC1c}ctP$q7kBb5h*FjcACmcIN{X^C`#Ehos-%)8K2BsOsLK4=KeTVH(Kg^^Kn< zuHy%L1#cvcZ^PvZo<5+g8zN49kzk^Qb?SbzA%N+8!a`2hzr1gM=U9W12 z)J6{U3MHm@L}HJ65(WEUWeBNy!hW?cDf_J@ek2yS4$_;!tDQ2zLBdWB8R6QWWvvoN z5hYB1gQv*ma<-1r^+`&5*FiF8=eJR~9`yqwgd=-xM`8kxOdpT*&>}?>?bJE??oV7W zS+n?E$7iD&n8C6N>Gv~qYlmcArqqKiYR^!fFq%*W?~>e_vC5Kf)V1S_-o z)`$3=P0q-H8*W=`DJ9&50@Q8G(Rr`bfdX9)Q!BVZ&^gnjp^zS(A-O_Rm~=4hoez;w z=94&`B%6qINDe*1{F*x&AxFC~`)%G!aaTZdf*ICosvUpOUwAo)hL+jVTqh^FqKl)2 zH3{Q!8rK!91F0n%F^x-B0;ChtdiCxN%KpRj$DA|YKF_RPO`iRnfw}-{fPbTDp3v5Q zaWs;sm}LmCY@2dxFgaVQ<4Yg&@Aem@Gm(CTwE)#uGxX#xpyh*{P4-(dTw^#qkggDz*=wbtSwBM*d3XaV@`}Rk0cE$F*_A_op8T z^&f8SNfk|0)t=Q9S6m;>of5=c9qY4m^xmF`@--FC{XUm8PHXO1^%m#|Gdc z?7~|x>@d#3Sqa5^+hZ>{oDQd7HrLy$T0xkq<%SLx5?TDlr?`t#~ z&Jt1T(A9f=0l7xjlb`WTGhw9PtNe~`Qo3n8U$pl-bH~^-J+4B7%6z7ZVF-p-=~JcD zUTILtfwqSKaeQvZ_S>tX-hQW|QC~QSeW|m(*YBEbbCC`+O>2TpQ?=kcAr@pA3Vl=? z$c*A~fV78+$m*g^Nvwu9+a&Qv1-*(wQ$zsESqIOXHvkhG(=YF3=Go8bX;&w#@YKoJ z3&~rVabHCFSH0%Q{j~kpJ;8J107SC5#cbdZx;Ju|eN>6=Dvd%9mdKp`JxrxyZ*E9_ zs6f1}aeW!F(E^(_H&aTa=Ny2KlEbqvnv55qVi6VKFD^{(sLc+!;_!j}>5At*EZaPM zS@HGdxx-@DM%txjLK@mUYLjOAs=S}f_aF1xa)5d^^AS{MEApacexJgGX}eI;R^kyv za)5Zt*J-W;lzF6%hCa`pWI@I5%aLn_wjU7+Mtu=>x+aL)f~Om=a3#7}A3ANqv!3yb zje3muyC2!<3r`qhxlgrB-}`h8eGUrV8_Vhoe?XyW@m$;L57pGzTx6+#8hy!o+jH+C zU$6dvP~aBs0x36O*UKX8V&MSY#4jx8s`!M_Q4~h^xxWNR!6vnS=z7vw1K3zr_iW+l zV`f%=LTudYQn3%V7f4yD3E?mH_BC!7D7Cz(PDTU-d-rm_loDooQ!eXULFH1vTRk(? z%_Bx{kB>D`Z==;kQR|0zF(5MHN}V$|tF7522+)`^8&Ig>TMwN4w{$4I$=K2(_dsMU zSb|f(X2j*Y{IPfxo^CCi_KYUvoZ7<=bk8Q_)a~J;%rXmCwh8SS$AZ&&HEyc=7e;uv z50)qh(7Goz4h6`QSr+lAEZ467lIJE>y7p~G<4uKLejh%8)I%BX`{%_sGr1-T>vQ_Z zxGk6G(T>#w1E6fe$39_Z2z*g8Unm*Pv&i3?M;z~lbKF7ZY~=)UoZL&Ad ze$6mx9{MSLT2og-0#ihv+4DM)LCDD?E?lP)${ehn-Lc6iGSTW(!=Y)P+FL}E^*QMb zZjgO~?@+@1Bx8|OoX8K;c;&MP?pY>1d(F*0kM*&G{#vouMUhzg=exIAkviu zhoeOcHWjIzcTUuCYIEIGlBJ(DljgilwxNZsV|`0`mV0gg<5})5GD|~Y1bZN`EU$j| z;FnLJ`UOi>(}*UE8mhW%6#P~`*VUOpPN&M8wCbjz%65O!0N8uJs@HY#l5)RsRyC7Z zcB1LNWa({wU6(&FGQ?Z^+raesDL#8E4xwS7l!<=s$7&>Rta|g-V)Z6RxvGJOTv6u3 zUHC(;CCj!y-nEV9->`u-Gv&|{L_MghVy+{(S%ReH-OpB2uz6N?TfShO!z10yIbj=9^{JrPj&RMryN-}m*p$Evf*mW}G!m?s zTpxpkkyWN286%N|oCgx&UtS&XQ}@je&_uD`q z8hyU!WO-a&qz9*pT0Cbwc;4RNQoHy=W?rJawKjs6bV~+c_HxQI=AEP{9k6#(qRZ=? zQ1`#dsBkW9m%7j&*rquKigWY!p+@!aBr^!ZW*roC4eW+2!z1pqxVNlI7SS1NR|B5m z3gsL720J(Vvz=SwqXw%w(rRch&dy2s)OZ|kN?ODZ))&%h<^X&yR6yJ95)!lsS=A-j`=m)-a zB3F7d991MkV7d@K)X(%kMTHlr3+qaX{HV1P9rUpS9Jv@GsUE@{9(gfT{cu)GqjnG> z8+rcS-XU&m2wMu*Dm=^XLQPA4rLl6SKEU|dz3x6qlBy7XF&E7r#;T2rtZr`Q_P1o7 zXQ*Bi^3X9&w+rn+q`Rt$wUME4R2TMM=SiiwZACydk@3yct)%ua`Ztxq;8t(W4kumsQXD0iwhx!*$MFH4#UgtD53~_ z>QK&6Ci0P>n`S_n)e2sF8X$bl{%lpvofU*hzc&up2tKUHOVUX(m0-lvPO{FOq~q_d zAtUHpHe@pElS7yzEqciZ=UXa^vQpP8@~7O#=3mKaZ=}o3TSbXBv*4P+9>~iX7s~Px zNS5xs7@A(EO%QNl+?`NL1d^OvU3kjBqZ^Fj8H2Ox$?f1j8e}};w8Y2=zq?tKI-l`LEFauhU~a zC}Pqb=k3l`=29u&8d@rEccKCAf#ik(9h(yaQvWrP=dA$?6#@P<; z3~{jLsarIdCN=rj(@dk>F-X-EJ!(_h=CzT z5BhH&J9wK|`%ZTV4p32%y=x3uxjg#-6HGB zh<$GI1O>kL9}nDm@`w)?6wcx|tH!y4+U0+PzzYTQ!8;TYzQh)} z3TE8@RQsE#0HhH$#c^M0dP^|j&#vkqydb8TQB#CLOHe^#RinSt=T+@DiK;~FW<00R zJ61_?!SI3AS$g3ZJrwWAgCbBF^Zs$vmyfEc{V<*V&1UhXq!kL^^glZ4Z@+*Bgh!+R zPX>%K2mZ#fD(9D6s>vd=ikr>nw_9iUQp+{RcfiVc~;M=9S9 z+FTVwCde|5izXTg;_&ctv2_o;1}O)_l>Ao>IN7S7-OKD;cZni1AkcCG%)toNcUc$?{rsiQe% zakm>TSo*#Yg{gdKY5u#X)^!a7);ImX)}7KJioG2gE)R07=fx&3PO#&)DCv<|4pJl4 zbqiGZhxRKQL=?#-^3CG?+*4j$6A^;1+FGUdm&;s5ZIv{CYAz(TJ{hy=hZ9BgbBUx< zURUarr$0rCa@wD6{yfGiW(6r`cEu~W62$(Ye)zqJ*kl-q$)c0G82RVRJ2Y6`ev5Wh zO!H=txWx(o3u5%26#D-LYTv)+S1X|_*QFJso3d4|RSK={@5y2#c0Js43bfvc?FD3{ALAnA|6`dM8i7a9>2~M^R6Z%n7Rs3vD$b z@Kf0cFyrdA2Wg*T4+;gpAm4&Nh11YANU2X={v#<2{RUuZ*V3PHoKHyRM}Dj@gRyQd zQrPDfF}8nXZn}`QyaT4Q721-x;i1`2R9J;_Ar2t=QT~2A2H>ms+hf!Swn&jDm6v*Q z+VNt)PC$R{WIT@bDtnP&PGeXf#$_U$<&t?1ykQB!KA_gq%3V@_9>mV;Sbo5j2TVX^ z{!V$sG3-+t-zOH4izZJuq1JmVmI0f8Qiq{|KqNoLz2@wtb}^CtI+V9=`NF%?Zx-F{ zRPmL@jikP5#?wF)X4kTxVueSj{y}Id5LJ?;3|F2*I`dH+mKNd8q9r~rh!VZSET-4^xu)t+!Ym%mGi69vH@Q8pG;hmmqlKpMn)on1#Z7BLK!A^GAwQ!a_H~+AYuoJ9dGn^yplN?)&X%3Gj+YPzEs2K|JFI5!H7sTB^bO_1zj^1MIR18t|3sVaKumIoi37fL zXw{;tFRKB}3jcgW;rhy{IoI0tOYq=?!hAB5^u?Vlt`@$5jdJ?#NB*#Zz{`AJ?s{rk z_0kCI8H@Xs!w~X*;W&XeKs((X`1x6{)WS`ImS1Y7(-=a9c|U<6FS9vx8hLGkwCA6) z1dca$>73$Fa44JdBTHOX03vfgL9abVUJF_Zfd*QCm(-1J@^NHQQu2BoPt~6m?KH~T z@5Ssh4+bR=zk-hcQd)mok_i4w8=C(HRqcPht6Sm(HcuICmbmI!S*h-|`6c-yrHN(* zcT9fY*GYO1c~@!)uW-Dpx9l!ECd2z5ii4MMK(q&u?(~&SBFXK9E=gfybYyGL#TQ&m zSnZAFu0LU%hS1VTNx-c^!9sxgTVbOv&|liPG-AjuT++eD{U{;(7nrj@!m9e&~}H>U#-LQR6T4Ihg88|UMm>=Lc;rl zhQ2snj^OFg%AaT}wPx|~KiF6=3Bh8b`rjJa{|Rb8lM!}y0f|`|%6V$ne}eB9+6^YY za9hQ8%+Kg$^PmXq2igtiPPn?-Pf2>5)A)HeNR{ex)nA=XjQlgOa3uIGyQ1bOSBOvlFRE|FWON)1v z&bG(4y-w%~F==_vnKhszaU}j-s>vY5tvjF1o8=qF6 zr9%Zoi`$D5{}=2k|L21LDg)UC?=Mz^+rMBM)GpmtI-y|YmCz^as@t}MvBS)tgW5Y^ zl<+|r3%Bp;@BS&;Xrc@gc`@nd%A69+lambY4-B7_@m{~9{U4iOsj*qSWyK7#o$LHD zU}^i}wvP$B9TT>g0@&9rSCWVN;uj0B<5i`ibK0$vx2f>rj?84)8E1HPS$8Gm7m z-NUOcs)d9939%$r$aGLq4%I$B`=?Gw8DMrXA_S@)MdJNxKlchG)|UU@EBqh41GLE1 zRW<-T?4UEFW67H%HICY|2%+_XC=4Z{ zKjtTde&!G45eYw<-BnV6aMFD~JZZ_T8>wl$lh$C1e5y>J=KE$Ec$bU7;>T05%M%FE zjl4{G-~C7~3f)&DI|(6ypJi`bMK-Wh&aiX+6fy=w7% zM}gGH3hT6siP-)dH_~}xl`eEgp?$ZM@BrsLx2ovPOqZfqXk%?gfH}F80_lk)yplxz*8v824#YKxgR0!S=2EGv{x>@Ez6dO2%JmXo4}r zVD_C}YrO=*SmRVBqQ`?}zMmLKqFKKrwn=}p@OuHDGycR~ zv{n-0mTo1=z$LiPmoa;*DVmB1d(rv9dgeC%?a}9Ukp5o!uUq>imC>}org-Z4myfwvad0_P0@hKkhQe4Xik? zE}jO_DcSRw-^oC_r*w=2vYo!K(wRe6;GSX&HFZ`hVeXHX__VJJi!cy1kR>#GaYrH~(HkU9lPQ^2x=4{wZ|RIcuil+atFlUregKzqfH|4ke_Xd~H-t$6(J|AX#d&+nei^Y!B86^2>ig2(b_*%;B&jRN_# fpY-^Aq1a1-rOnPaPq!`nw?7Jsf}&KVNCyFt4oVl4A}UQq1f+y2A|(`QLWoKWA|N771QbO?5D2}8 z5_*#!X`v&X5K4fMoI9xK^St}~&))mXIp;&JTuCM~tIw?8T62fq)YUwB{LFDOGP0A` zuc_ZABReEdMn=9tbrksKl+cO_@J8-_TT_)RyMuiW_&{NIMdu0`89b62Z+Qgxe9YyV zu{#+VO&#e)-U$9|Lq-O^cU}F;9dGlcN|QDxJD(LzVbIjG8%8|mnPbn?+t(hsRbq78 zOx?hct(l3Jq3`ODsrti$(-)1Mddt{Uv-ra(-Zx2cvt40G4Xb$H@OO|cHDyJv z(vd%qPL&FN`K;Ui`?ddk-)w6&|0KY?bPGyvb`yBX1rk!}d4SO{4c;ie-|y;URe~TR zv)%V0hT^*_gsTG4%{F}B`w+X|=9dMN(QR$zJW+h=V;fxFxpHGRQVH5cpz`@WTF3h{ zBQKlid2gP10DLFF*0{qB410H>i8MqeV&e+0ChYPI8KWvyS9qJe(f|6BUFVM>sUAW1 zCmR}j8jr33FUv1`{O&aUFgo6s^3?zrkNDhaVgXA2ZT)1s zo=~=sJyc@;F0u*kZ7sfP{hXZ#nb#a2aI#%pl|t&Mk?Cq>yz!eRdR3UV)Qp9i^lQ4O zFUap7s%{H=^{Q$7Rey7|prN6;&-7h(?3QbVahc}}TnV;AQJfR)zq{lBokWH*I(QMz z1PXG51#mfg6r7f4idgSVzEmDVyKxChzza+rX&PuTlFGf1wQS*h7wXX1Dvt15VybDM zt#f9ZW>kHl*TC+0ZHkJ%iN&JGJ6XZwVdb|q;#^PqfIoj!9j}9P7u{tubs7}!ig-*^ zM36Mbn)joPhiYSEW?e|W=W6gb$#eSavLBSMCN&97KQ$f-2nxSC>`i=4XGOGWs|i+* z*k_1ZFswp!U8Hx>^tCz-)EMSuR~N!lW$EnyG=!x;j9M#ty)u#c5{ze9g3;8!Sa708 zJYl?UTS+_62Qv}GYDpIL028hd6=2MkD{bv8@X3$ONN%uZO+bYQS@LHUCg$5F>hfZt zZy7x0>$}Oq2F|)wJb`ntGKv?C?ItlI0>)bnk8B&ozhYH>+gTUJz^69l-9^DPP2g;${}cL2DYBvu>Lw7&GE!SP!=` z>4+;hJ}u;+Zc~py&q*%2MS10W8-$h8I;7T63nKz<6-M_2pT{s{9G|TZzGBoeAH+Gm z;c+NzVd&k|(b$Mm!=)*aBl804CZ7gDBx?Coe^D#`jlfAStsa3;vXQ0ml|_VETEP&W z*C|CDPf-Z4+=#2f^NXrKImXewu|*lf%@OvS>>cou=p($6r*u#U{Egj0m5Qe$BkZ1Nz31}jhtj~V!iDE+dz zp#UQ7NzfN43fC4esai^G!}Gh@L>K@2OOsG`p3P>|9(WO-{$lVdY+8C#uCooqtEn9U z-$kRg^;iqT>00d;U6T_K>Jjnm)>{ZTHbv?o8&7N}YEnrW8p?j|o>j5!l~EN* zfj9cxCosrcxxE-6HgrH+K05^(DO*QkN$Vr0SjvgvNyOK;hN$3v4V<$H+gekwW=T&X zn<7!@LlM2z_KQm$(pa#lTbci)RnzKut%S7KX}v^VwOCt^)`c_Jt;I~UL5WAke&rVm z+rQb)J!`pr)mF&;hi%50+1>M62886IIcBxO=F716|A(cT8atEGcv)qA2R0t{e-Q_r z$}5IAOR3sDbm9wa`WNGoH-j^%k}?31@cxTY6|3p^XWuYO&p6Hl`*Lwg~4`pn?T;-x?(WyNAc}aH-DcJ$PmlJkK}3%4Ku$yUw}Lx03&mxe7z8 z^rvs3wh4ztI$#V;c|%|X1md_%#l;FxqouB9Up2_B$cAJg>NK;R z0oU?(5-g&dSi%1!6Rw0ZQWcD~rs1~AW6o4#l!slJZ$)Rk6mlbT4M+(!4}FVi_*><` z^42oJ!20nDMJ`E7K5moUClpT$SV{Pg?~YKP3uTYc3bT7;hp}mVVn)~9fc8wNP+2xT zv*wdOHqil^w;8u$UFv?77Fpja`9yJhM@b9yjBDt9V4B8Ibo#WOafL;7iUMj{e8R<1 z(TQ%MmDnG9S0FM}68~k@Uj4@0t`t_qV{KM&#&#q9D`Zb=NyXq_c6Lo9vKT_06i@7( zD?4mCzGo{+Xlxh1FA`YdBPS-B+%>|aH@w&ln+{6^{m^MLk~3L%mA<8JWTGv~9wy}K zbgNJTiRG>B=FO@XeadA?f8JxfdVcoE8J{=TB)cvr9dnq&#K~2CwSqvti*f~&-7}l# z)}2!_=2-A_Tm3M6mqwV&@+1fv=HO8`Q~lF{-d<9hW!~6#7ql8QbovSkISDz&ks>bR zaf906w1}xe*hjOqydw!3QW3WM!Yh`lH=|!`7!jWm87@h92iNFhvZ@<~P8<5$PP(yE zPhzK8#=DPBY9*|{0n-4WO)eq74|guHv2N%lF+2$Y+9*}LVcncr?q?wapsN>`zj zBx|h{sopF1s!ocUD7>q&?*;-qLdQf-_K)k!L%K8<>I zhS^R+g^^9f(e>$=oLDm5$lk;vkG6r|#FH4l{>EWZ)6z!)f6=#K2(^5L6V z%uDHHlm0Rfx&i*O&%O92vh76Rj~jQ1VRIt&FC+saI;j&CL_Ug$c*I>hVRADrH`>|< z$3f>nO??>S%`74=9L*8;&Yy~6S7rH7!nwkNFZ0yOTtfyA-GyLsX`T_t$6fwYS3qlu z%AP_!pT9u0!7YE*p$RIhiGP~bdD&ROcu)LO{weID$q$@guDnIq%^p;NMppYJ`M_~h zUBp#WrDJ+>E9Qk%44(>c4E+s|J|)K4klXhJxbf@^k(uwDav9)GS^jChZZ5@}qSG9m z1_qFtHS+Evjx>SyYHx2VYMz58=AoPu##^T)w{Dw#Y9QGnanf!A)^;K|Q1?f%^|nDZ z*_jWF1}C2gusGiJ2bb)EEMxuOo{GKpQx?3trxcWF9cgmOq>_!u_81lLwlo;(WKlZc)>gq~ zCPUtq*Nld#xQ3qLMU@F{=>A}TbgP8KrIxI_lWz%d#w3L%n?!^JliJL4x%;f{%IwtX#2WIV2a_w4(#qEdCdM^6{cnA}M{x`w zJ+ROfjJ+(9Uuq(!5@PW&Y%ElfUs;#&E^{W=RStW@$i7H?R=C8(j#=yvJs+LVE+-5& zf^I9m5}4*YX~N8l6e@H-=Ah=lxAQdg%vw!SRW&j8euk;Va$1>o`2LRB*omZ1U_Ei= zPOttuXNi8xfQrDqZgMU3(VCnf<+;X#AYr!qTp0Gc`3(bS-cru4wUIUg%MZ zW4zVsC&wKf`ooTDh2_Fd)P)`6rcq6G9y+zNIF+l@29F6qATg=u`?<25OLi5sgR%&5 zF&)D!w}0R+Nb>M9$Yb}nRidK?O<4Xhia@gR3u#HmOs44h=_m4WhL*07vRDwZc8yt^ zyFTgNwd+$iWm^`oMeCJKLY9m#jtPe`dsViLtlZc6IWaryjUTvmiGNHSQDYTBXF*>) z!U5%bD>}nZZWGWIc>Nf@GIy-xQ9|3=xHtiiKI>Agu6sfKm3Ukl`mCFu{Efz|L+cIP znMB((X~SST1!YlfD!9^5(=wk@cclDi9~E-WP40f&QCaYx|r>%pp2|Z0oFNoH! zCB=``&p9K}lkM|1m_^jNG}+U6JD|G?*0u73F|V3xoc9)c66r=l?L+Om(h;z_4_^>J zUB6#KIsn4Ryz)hZlPZ8qQEU@|##~Xy6>xL1DRX(g!l@|VGfs_c;t@O4xWaPFUzif- z=5u52H}@HX_1ixx(WpSw$rMh3KqJom!xE=k?UfAt37>EIBQo@=dahA&r@~zge);qy^jEQWUhbbJeeCjYYn_q@TLw#Xu zH1~3&*@uS2v|?Uc&eAnw8P31zYVpLJvQ-5`IoIdtp8 zstQLF6C}5u{k>)C;SZBOm@DindgdU>CZ1Xoa5PY)|Y96Q+Ge7kZ7Mf=e#m&*^Iwsc~b{JY1vGDTD!t!_F0 zA;(K^CNdS34o$xG>jbO zbX5QT!BP5mh}R!`(n)D@$SjB^E_5k;+^AyRDem)LNAUExhN`|oFk(O~IFsS{=*U24 z;uZNNspHF@-_Zey=%@#^^LOL!XDm9+ANDuja)P~kr(8hmGjWyGYP2Lj@mYD0a7 zQy(6RdMDy^uCKj&ecLbwt^--bcW3vT$BJ8KL{6JAu(6x4>Nws64S)UW+K6tsE z+wxFveF|6Yw|@FzNC|@^NZ4dJA>&I}p_<6#>-}Q~6n}y}D&Z-e87nDcfu4=E69Ftk z&+(Y|l~c^SRF)I9$3-5*ht$8SkCD42S9m@B#uM)^Ox>@_^@heu3Q#7a-wdYciXIM7 zsxZ?3UK+oFbMu8ovIYwEQF?J@t80f5wVYtS86svw?kD!c442AbO`H4N>W2Fn6<6pq zs}^7!%!!I-6cMy0c_nggbBDZ!=p_d~=@(vm{H6Bwuxoz7$=3XqCmhEKVE585_T-J7 zEjJAsejM!@SsTw0G?~gBvhwl8T3QVT*N0sw?D~1A`o$gUQ}0~8%zjF@l_l7BlM}pq zN{$YjA`N@vvCC;}Dw6plPY$!d$)`J9XdfLRiZb(bTO@WLGP(Cc%bEYP<|zk*mn#gO zZWSz;xYP&KDR;Gps=wZe6*9{{A49v(5VJ%4-$HD3L?5EGVx4IbJ49(za$_q|o$ey+ zkf*!j_{Mg@^+xeLgO_^t?|GMgnBRNx)lsM1s7PWUrvCGn#5BveWja;(3#SEZC&tdC zK0$^B-T2#PF2ycs1&)TT>8E>Svc%v{q9bB^&glK9sxrzJ(bRSsmc4(l29lD1a2^-# zSnKTECJPT)*0^(*P&FO>w& z`|Sh@ip-4I$Dw439$*K=hhK+ij1J%M`0%#GN75p!vhY~J*T|sBoW|}K>O2$J)iLMs z&nk`mG`&G)=P2Uzgg-eM;Y+NoXKMz0AMTU(_lY_gPCS1KVjdR}QjcQtc|qK(Z6+4=)PN|WE=4T!n62eMIx*Zl&|OV6s}cG1^Qvvb zXMM+3BiZhV)I(jDP4SgR_WCIh7(4fioWr(te)#yJD|=KS=hk1+%)0zMaQ-2J9uzBi zga(&qp$dCQCk;DOhfF{xB=R0axBWyKifoZAswPK^#nYvp8ImvH5cF(Au|6;N(6>Ga zX#0i)q{u>4?D6-8s3bj5?c;8-W>x3S1QO5uYdV|rhbTYw%c#Qs4FfdYp`g+J>J&)# z6AJi4rv6nYKI9tfKa=QxU>0_rE|UqL?e=q(ckh_91GqRAOR@fr~dsVF}105$BRqhKR}-B$Hc>pgPH=-q60I^l*}3( zBYH|X&Dt_$k|l(RO!Vk_%N~6zkC-cf9%XfEIr0Ie0VR`KcGFYxs_aq;hqb9(pKYYVv^d$-bhbQFw~F3LzDK zfN~2G_qK}o{VTRPXHkt#9>xp;Bf_RfDuMrhF7Jxh+z6K)zDvwFD`tObEu2~Qy@q3o z@_v1S-NMRmA}vtO=gm3)fNzM`VxuZ<*WxgFWB!R{Y~kSDL^~NC!%Wc5q-L098_L6X zkCOB4df4Hu>wisP>RsrsloGNQw%yL zO9$1b_dkI0nfG^xkuPAJf&lsbZNIi&-dS;)>3gN7JJ#h@h}`J0g$~C1SBIy^r=O;B z441U~I#1A?D3_|z+551)IOvVYFqP^n&J2{hEJYd566qcKMC(EbMfs5`Lb3nmTzDtO zj8psx(%D729ITHc$sc}AtjnYY!*#wQk5(9H}-#wa`4?+%?fqPJ|E zm>2dyog6tWb}Jk#oJG42_XEp>sNic=oRZX*ljH&$8!nl?!8%p9XCDP|Z}~qKGNZvP zHJA=cRgkGZ347?rW_2JNIizfxIka&c5-8>W5&IRo+DD=<8MKEh4Uhbx=EoILxxtc3 zHo+g0`T{I?upCN7={!g0IFCP_q<_`><*P=JkJ2IDInju&OUJn{6@kM4F?P`ihi!Dg z(njI7k8*m*ct<5|^=K-%!?qOqD(POlBAG$F24T*sXUIW1L@srf#>9D^E<-){Ck%oCaotS0Aw`aS?X>!diMw(N_2mBo7 zdo^b+MLFScmiWB*(s&gEbaa#u+9<0WqP8ebiWWtKfou^coz*w}9}fzK6p2&3t%cm& z$qJyOyatp=cX^aWk<-kggN-RaFtV17W}L(rH@Kn`hzi%jwxL>m-f#Dh(fY#_naWPN zJjJO;60aKg^^QW*Oc!h-Z1eD#{e+jf!*_1bs{o8Bem5Ptb{61p8DO)Igz@yQ{o149Vr z6YC!KYtw0(V5m6JRU8GCH^bj62fe6Ye&oY5_TFEq=-A&gvztD~%!W2L)w}w!VZKxI zEwB9uUQ(J_SFIMB_lZJ>dIV7`FsA%MfeSek~bvY50&9==|fh8DJ$u6xvC(!zau7Z zO36qkTcpNC4EChI;T5hq_M`tE5r<2rUl#J2+1A>MhfVg1enH?gp~chbVE^fN%!fAX zifIq@Gga%WfiLZ%UVIap0$vR}Fa81RKWNZ29tEx_>B|Q6F6*~z@jDo(a}K{_?F<*o zxxlD`9ur@wH{&FEQ(2tEe9)72+-A*w>7P`ZQ_4n6!oOnOCSt{%{M0#yQS(FXAcas_ zSH8#UG=yW90gBoF=84juwXv~*xJpedz?}9V9TrTMX1r+DqdLYv`>I5lQyE2qWe;TE z#Ir27>T~Fl%xJ{EI5{C__#z!t)>rQi0fa1JoB|FuIC&Sn02M((l<6&6>d#Ds!eh^-pQ^{%g4gKK{YQ&D;9wQtDWBCorI8pSkjV zJ*VFXA_Wg!7Dek(y{YQ3(~xpqOwf}Z&zPSvhq@#RkJiJpNTR=)<+U!4)aW-3fp-mMB&^9oUVP*TgK9 zx8qyiYwBJT5hzkI3dfF~mdB#JWRz3IAM=bk}VO|*dMwUbyw%Yha<2wz=E-uePFQ1Y_8Z)Y)T#sAl=y%3Wjoez?OqPD0j=V0|D_fM}gMgb2_`Iz`(xB((f7@c1cFB z(#;lY~&AUsV1)W#Ta z)CfQ%>v42jLCOoN#i@hP<{OaFzwjLMpTFCU=`g$2SfQ#}4y<<@@=!UP*uneYd{U*=2lh`Xb%Qnl3`Sv2VR z5ZFw4t91UT_|0yz-?)*;;W39zLK5(!;}*bCUM;@r6>oqm#54uZ)_!Sp7Onl&VIAM$y;wQAA7smkQkUw zHtPtKil5uE_k8MBW<_c)&E9uSzlP3cm>>iE-<`kcY1Zhc%NeHn@U)v>hK3z!Y+{Xn zwn8Ne!%y05$Q-#b{pPu7(Cv|#`|jNNZ9h+KC{trAwqZYPd!|En>3#R!NTH(6y9tDH zBLCOy@?3*hK9l3_Sp)Cm4yMv_JRJIV5B^{L@-X1>CzP`OImX)KiK@oI+~wwW9dC`% z_*S&zHD=2cvl+-s%Nt9b&Znq-CW#3 zH7a^7*HRyy)QyEvMSiRe6g8FxDOfH5psH7zM^m8Trg5!G|2LLhr{Pv@yX%U zEtGtbU-2S(#1r1;UR~^pmipC=;clCGlF(|H!>lO)W;|My>)NL7$S4+_528x2z!Zlj zBa|z2nzSbbu0EOkqO_fIgvHC^J^TonR|^>~|C)*1+PWHdL7~|sdCsW%+7Yx_6VclWtJ9h8z zFZxXUPOjX&=U5{?7`SEs46L6kVPY$vmzHag6OP-$`5W|o8vS~zVJALZd<w-`!UYJCOTv7zb^St~Z<16kej-EQ?cx?GT6 zCQ4W=3mV13v?F?1Cz_k!iH*zcN zWNol@OCZZyT#6%R3>y0OepFp{Vi0EhyEF>@!;+I_6y}XQAbSzM3S2D_8 z-_bCAaLIpSHYX^vC191{W}xj=`GCFtmpTS-P2t=rt1jJGdV%n=rVO9_WhI7n(_hX# zsK&R4Z$?0zWY(ri=D9|t;6oIk>sbE?`jWhkXuhwn{&dt$~SRk7cO(sOYMx0fOi zixR6MXwPVWQQ6Q}DcrQSx}SGcz+FH;@uN;P>SLG>l;GxA8)8j8%cIh!4(p5#-3O=UNw5;_`WDFig}K)Q@p9@a`aK7k`&^03W%$5HP`&Yxf}RGvaoQLL#%wfklAUw^`y9wM9U=lE4WY-?_zin5`&u&FY z)k)D2*ha){*Y`6Gz*0)AlktV?5^KfHDFp*zivr@plw-5ZQ}LAcJN1IWlqOG#24|Z- zRo_kU9B#HuuO93yF1_M=&ljUI+dr$8=vLv(C+4C186t&xILU{FbnPT=Xk4ayqq5X2 zIqs+Oq5Z2)%}x^7AwtLWkWbc;HUUug7%IiOcBr(MZ%P#w2DDq1zUn)wo$Id(I&Qoa zeLY3WAhg|VPCr&CKk{)5(t~UB){LW6>#uIM_LwZ0mrdWfAT4L3dLkgbTD*eA2fIM? zLsD4pO4nwPUqWeCo^^Bk^EcGdyvK=$(%f|FV^uSp-Sg%DisML`mwiL0&c!&KFFJL@ zBQdf4Y1=_CoCXe*gl_iys;Eb?t~xQN)qk&nBOEpM)~s0@WtQ% zrHLZpd40Cv-Ql}zd2%rO)h=HI+{4))n=L4-oY@ziLKuwYvee*&FWG|wuw#45ep@Qe zel)abn%^g!T(zmC6mP%GJ>0bLr#ccE*1x??0`9^)AA0oIRaH%IEvrBQT}k0c=wM$J zV!bia+3&8yt;xL#oYDn9Mbb&=yL3XN{c}s8IS-Ox0^J_fQn^<-(R;~c$*T!*ZA&MS zz`#6MOsFxGRc`b;_;9yW>DWbz zzyD96=Zy25oS7s6bPa~u$DqmnYa<|3N&m_kjNSX=Xt88!kN5V;6vU4XV#sqa*YDFC zil-*WAMH#+C1jMX9#+gJ#d$JH@GT2?lb?0-#%gkYe>f@bzgNzd()W0)Vb@ieh8}(N zk1lvdZ-&mnF>|$*+yv9@`rdMPdyT0z2j~v!E*rKWu?8YH8oB%EpZ^)(Q7jT%=C7Rg z3`NRrxh1rekg}%jP}2$Po#8C7%ZK8v$*Zes$YS!}nbsnm_xLW+Z4r*yN3Zb5A!WVf>Q%vM6T zX-#Istphrhyb+Npmk=0w)(TSc_YhP`w<`CKKLG%O3O)BHtr1OyNeB-BwkQ+m{r`*t z8Zyft?l)v=mm-t^Of<5v55UwBnA6tZiV(eKOr;z7Lt921YazTLR_DbjdVhh1s8^vY zyCtNidp~3^DPb3Peo?OEXJ9^+%6zDQav}2fA>QP?b_Ic1&~t}iCtGDT-#daoqtfa6 zSr$PPO3I~kRDKKCF$1Y}X4ty=WQ-MO@5pWqzvhY;(C9gegNXo$%QxzGYBd8~zXe-# zXTpwrbZ=JA$q#9|I!hb%5NVeDGv8`T1+bq=GKEuqd+^^#g;pRawhH2{0Hg<kc@vG}B+iobKF(<4NK-*|mE!E~onwgLdg zi}A^imni|K)&P>($K;RB;dZQl2dP+!Xz^$`n5}XRS_R}FxODFcF6tEo_EZb zL!SLfWl^tSXvK*JgG(~Xpm}99MW#|w^LwWhaR68i zmWAM2*#P*{oeN0M`fz0jY|l_di^{SN4Ma}r%K;9<+FSWD+nJn6{@=r3myGKmvQ%$z zD7md!*tY)G_EGcIAA0~u;|F-w!@qcT4+Mq7PqrCRZJ=;&tFxOivUi~$u|E-w1Yla* ze~jG~w_=Nz6x!7?wi>+ysp46d!kKZB@>`>Z;$sJf8i5p4ywUKnwfxRm&lw#&ke0rw zKxj8V6JGOaZLG@cBLE1-5x>d@@e)jtk}K)(4Q!)hSYRDdF{mPy)<9} zS3v-}YnyGeMt^^3O;mmh0r1Tf54xY};zu9?gCaONK%5=`oWk|eX5b3R!(%ZU4Zk6_ zc6t?zTY%)&Nti7O5h!V`i@v-lkc+C?psgx1f9V5_5VPK>D~1CYFF6Q&U=r+-QFSL9 zLLTK$z8YU@sHl&RbmMjOmZG7r+xp@M+A2RXe0XB-#*8>Pbc7U~%^dZBsCDRXL9JEB z`4{;686m#K9p)9RicYg30OAt<-bJ(YAxf>rIUZ~XVD><6dxezv)+0{oiiD6bWjJA} z9mJ4PM$R$m7SnV^oI)ugNBXyRwP6`G`ijF2r+4LNRM6E{ty@`EELEzV&ME&nQmD34 zo^Tt+y7j}wo_g7TV%yphW&BMHB$fuCQ@lQ0r{icV(eH=(W*1)k4u_0ZUvgm$da@Qd zKAp@1D_{($sR7m36~>JX6{pA=w5_87LK1xRcN`e3))&XjsC{?+5N_nB!MWH{Fy|{$ z;qL4k4!3mtEXj(U`&EJKKdMu;6uu&KGMou%`9fxz0mbsekH${zMRUC7ugXHM3=$8d z_ZNnZOj->4(R6IqH_wNObmlc{gZ67k1|$n9Ltx zJ6bXca}M>VNmaJN+!g$0O@g#tH^pAnkl>79YoTPkDy!%C8;5M;A)}XDq$^JHWyo(! z2?#`;@P>2va7pk*+@h(I)F8GjNnxPl}rwqb;L5C%|3 zeHP$*+{*kU?`&^|{sPGPXtXpVcBvR$rCf9@^jP8}2_~CbRvDhWvF!?&=Dm{fS!UO@ ziz;iy_-k?NuEpN5$vRD(Yux_YlvH}0+Jnj`rdMYVvAy|xofT4C|Yq&=@lxq;|qFb|*xdD17W?)^rbr?UF$t1h8?MnW!wFK^JYdccIKElLA*K8k$EKHDVhDK)FizDQf>iMXOcpGn;uJyh@`y7m9*e zePSY*Ys?uj+Wxg5etc&4gpV=DwE(x5ku|PayQi3uGC2ICd>ReAyV#SWcbrw?&%%$f z>Gww@pI7NFDk{W|!M!~rPOcT?nX}W@R7gBGV&~hW&U9c;Jn9o&Wd})yZlWCv?PZjg zs^)&dJ5D9>hONGCj#u*Qkrr!kjgHBl7juR>Z&?_*qPZC_M2k|tjdTcOAxtd@dTI?2 zxdaKfSt3in4%xoIKGku#a^coA&&lpQEI7=#;5JcW73(n#g(~#{a#n_RWPlk52f~`d z_OdN{lRTwdIWyLS#FTT%ujjnak{VJZYU4cV$nV6*) zaTnFht^{Qs(;=L>+xbQE3P(DxTXmSNy1=+K@j_E0m)^$PoqbA<2hwHiyYxdIH&msi zR6%I_3b({I64dQ~C@{V%^xYTGEWK6^1@AfCrkgR8<4#tQ(US2)cRby)mhl&}UhJCE z$&YGGJpsE@&^BIe3nT8q`#i4mQfgP2RK96YXV*_Yk0?=`#Zs>EnlEa;6{dDr>I6kj z+fQ%KR^v~Uvp>Ah=aJ<;-?N-Zz}(_SJ_UgDK^MuKqfvFpW*bLBk;QjpNwQzW#E`8> zt#s+*as3-;0Uvk4WSz&_uRhxO7#Uft)p4x~LR-TM@HP)^FWw4PnKLrB_LeBlfU(Ed zK7)aVS`Fx1^h_O02xoBH6^@40@$=76M^m6m>6a66GKE;jrO8GLBX32ob%%walIW^% zRz{{7kV1N(t)GjtC?}Dzvo^Ma)&8D;wS2zhwePEZ5oy=GK}_fR`Urv}k{ua(<>0@( zzmQ&@KR>jKnn^%RKu|!aI)!-58iW6MtYrIoH#bDU7=OJ-Kt@0&OG%`IgSb3JAhUwBUW-H7*iR-0q8yB63xx^qw5rB9KRWPgA zV1#&_UVu{5KwMJZSLOmfC|4vAGFrqPyMtCB1gdwS{gyKq>V4X!J541f$s$3XXCI1n z^Hy;51Q$%-obByyzXlS{CSCaJ56;5h2y0_#dYDr5vhnmv8F(W$5@IX9Ey^*tx}(TB zG_);j>4V1V3-*U$AH%D5=Be+86-StG{Wa6C=%;qqpFv*!WuqRUlh`?*gT_V`elx~y z7U$gMBI1IGR~o-Yf`OjuOhv+8xUV9VUu4CQs0Xbue=A8Ca79eM@5o3?j5^qUA+xfw zns+#^h%oyh?9!$?yq2c(d@0oagif2DOvO^S zc-a$}m896T=+KBcq-QS^ftxRu{rXn3E&W$Bi4hx#5Tw*5JbOJH9V2o@aN)lGhdWol zA7OjbhX2`7kt;)*A7;B~Z8pb-5XR5j+40N>v-N+vn(-o!TINE+_Vu54B%O?A^|8E{ zrHO>ske+m%9b`}yCcXCdd{lk!koElZkl?O!*q=#XuT8qTIG%sqf{ImkeGkTG1dSpzWEU-S(8@ZyI z@xp12k$I;f<+xUcCX+rmG9wP0vRE!Eg^dP`TSUrg5P6ht=#@Y77&E}ujAd#mQHnD3d?5}7*q)JB`Q`X zB?{VdnZ-qK+>^7a>7r>;MQM7bKiHt>d;6>h>f5u{vtRLa2Lg3(7fC;pd|-)P(h}$* z(MwV)(-%JMezTbo{CQQj@cnC0Qd8)1QMtYpl4e}X-_I2OkNNm^maL*~$7kP7J(Zmq z)17d}V>-;g-00(}PnjAwEX(c*CB1KM)BEiqLWpeMbZ5ihRkM|v&|hYta(aHAe$Nwr zO-S2N_}1}(z8iDe;bm(Y2s4q}$9wvC(zt$RC@CIeZXCT z3(Y_ry0Ps+FJG-ob6+(iJ%di=3ES2pE-?nfr)`o~wU^)OD zbg;^QM?ae<^osVcV-HqW3W|(MqCG}0qGOwj#9mJr8Z+O(v_JS@mfZ2?I9xdyu&5iZ zc>L`(aH`H*8B5+YIsRgyTZUB!miD~o2goRr#H`w@gp|9cq=r$)mh}ufvtb22jfRHGxDEz}lC<$j-E4av|K-8+o+;4JpGYS4^1%0W5|lTx!c1ld)Kh>0 zYPR$Lh&@toTCPw}da{S>0{+bpMz8tazOTQXU=V4~C~zAv)kFr5=>x*{{o^+iKeIp? z+nwHgG3O)@BVcG21S z`MbX@+G(QT7^4v2#X6y7YX?N_8#_rkJO?2sQA!PBtRy^=LRdnKd@MO(wf8@>lAmIf z?v1OL?$#J<+fI_>x8k_;Kh}f<&x|GUNY#VEognP`)0goLZB$&eO)z&G98m|7`3KBM`1V6fUB(2kURVuJtIq zt)TxQ+pi1E9QL~QQQ2RZgBe2A2kaXgPD8lCBk^u;x*!h@ zVoZa@z?y8m!&*+Nn0XS2qr8DFx=d=zXR&JDb(g1`qI<*xv!25a^Cz(Z87x}=IRW_p zIDveu&$UmZociKu14Uusug`ea@O$y+IE z`n?^p1(>y>PvtZto|k!h_JwLu?QjaXy|DV z%shS`b#XKvDb-r!^ze1dxpE7&=s)A&`0{%nPu*uGc^q8+=K12qS32RhS#6me|KjCe zy4X?fD7TqB_AcW+)}RRTHZ$JtHv%R3jV&_$dP<{DoTEXcN@RG)fa}me>GSQ){VwL; z7-bxxcB-wGYoV(jX0p){zW6BQ{KEUx?Vpd?iNGp9`2W=dE`Iz%$+f9?)Mce+Eq=BC z|Kn5E^I3C0dymOF_PiR(;^PM{gESr!O0m9)&dE-b}i2aNjdQ1)lsH zAH0F@`+zn#!({J9OanJ6Y{~~SM}1c2qiLZI(yeWYM#aA?AoLWv+8|)Qyu}F0s?r3f z8Lce+-yO%aai0qUfPV*H5QC~<@fG4`^$Nf7O03JsaJujQZG??{J@NT_Wpq~1)w;O= zyJYi9d~heYT@*}2PA1Y}kpoA51Jg6(0@f*Fhcu~pcGgJ6)r-Xy<3%Ad%KsP>s&~tt zQj;o*Tv5T_c|0RiesbZU^9y?-CYw%G#NNL-iv-XCY1Ittc{83sqi*~wAn{UZn-@}*OK>fyMF$r zVO;-G-nMWc>f8M%OSL!Wp8HD}^BR>|@ltlAsDKhXzUSK+0=-Dz*`3_ydow(S_Yu3m zA04k|;@Zl7T@y%t-+g)yWiX|l4fGv&+@s=*JofH9kNb3tVYUuT`QIdWMT$hZ?=BVR z^ppV)R+1HC9z&_qSSrCdhrhSer_yS@0ZAr9k-4fEAKzrU__8nQ_}R~mtiaDPs@8eo!q4^ z0knI^+r>TV1Fc!SzmF*j8ck+UedNt^yU-%hH@b;@7+#`(SimLs+>^`5L|HziQ#LYj$=5=eIDZx){RWE?(Z7c6{Ro=|H-# zT@H~Qk#&(p?PZZ+7#-rBTHiCLWhZr-xrtdX*omR1S1dHXbbnklXXm*}r&Oe7Z94(& zuKvu`*c)dZIjwf)V?B{&T83wurvkwRBUb)^uOq#GF_L!R^(O+ap1Q!lT)tA(w=n=h z$TMhJ{M#&;lnA(o9t&R+1Ne-_n7v=PT&cbg1YJWDhWen7v5B&pnim3BAd5uifZUJU z{P|+PZlc(Iex#x0IBvEI3Zy>tNoW{~4+0sDv`nW3yJSG?-~ER(KP?#(TVxr3_N4Th z1}<#t+aqnn7M$e;Ae97R7Rw_ZHwdBqQP8NYk@R+%N5bkrENDV8$ z2B5wtD8h9Qb`w$ub>RnKE4Urb@s^*89p(Xq^p51N%+7SZ8M4fYgjs0G<6S{cP$dx0 zBQAMN|11?r88`QWXj(8gX=Nk0iewx+(!T7v4;>b$753mJh zCBC$k`mWn>Nxs77b!njC9g!fT&fl9EKViNLoI|>A)3_GZY?EiUWUx}@$02_RKfmu+0C;<4HWKMa;F_mc5yP zkBF#XpYQouch_1kB(^x#eES1`bjxVVfd_5aSdt5m&tw@2gUUJl4zKT68vr<+4HIQ8{VSHPe-#e^jC9Ek4yj*OkBv_1u=IPTl#YohEk7LM*sxy{D+Q{xGRGzW>GB% z^r@;A%bybY^iKi`BbzTZC)VB+V^i$5J+N7M1+11n57gQ72i=S;7YYtJXUoxC=Z5$K zUKKTEvDjrhK1XUXDDCd~@$C7F?k>Rq^dB_kR6fG_RU=T$J6yN%Wog5=@)S$eiYW{Q zET7OwN+Qa#luD_wq?KmrflG}+-&zmO>jH~_M5x49*sa6%{6f{%z0e!6RT5Ssb5qQ?7iqP)3vb$HPQ$86DuTJi_%>QbgHn-$JOIph4SJ-0jMJN4 zQDQ(?R;8Rhrjgq}Mb7vIdt4)P)TPRDZJ?YR*5Ux`-yF%{EEROmT@CGP5_T7K-7Y?$ zwLsDN0%3(HT8@?Tw%3JZA8HTMME-zSBnDIv)FZepdp?~cL)f_9jCb=}Kg4L?_E89D6rIlL^%`aCe3idC4r+# z(hK=cf~U~kG)bBRgW4ugr%GIvf3(8?i4%)N9UonlLOG601~H2NoG#9q8VpMWlq`r9 z;ARnr-C~7R#T!-rNrOsF=@MAVsU+Js(56vf_=4AB+)LIi<|o>RzF?L=bgJnKV1_Ce zrowlB8Ut+Fw})u}&b_r*w-P69-)Elz8wfuF{-L%t)mP^f={Kw@!rO(P5M|>ac-oqbv(Xf-c0<=04_3+M@TGy;cOt$tPN!?v}%je=CB`R&-Ixm^5s*DGR@3BQG&ly!IhZR z=YTmA*6eeQm)RO?-C=tn0+`XC6&Qv->;22HUG|A{04C3%fSM4Q=M(ix0rD<;g~?{< z&Ft7bDvT8ir-D!en-AHQ{V3qw*&LvcRPY7Riysh@r+*XECLD?vvib8n(a6<>f2s6sZxW9RjIC6@E+5 zvO1)g|5z8U=Klp3!pJz+H4<9II5adrSi2D6JzO?&W1KCAL)_n8DAb-~O`mUfRmNKi z{a`{dYlrm}US;JJ1H7-n&0qOs^dj<$17H%LMTlI!X<~rl%52YqEv!oS7tI22>IP9UA0*kpswS>&Qcv1YC8{o#44PT(QKLJmE=w3r$jY!U&VHU}k;UBor-?gf6l0y7|I; z>^WS3^@@oA-I;c#^am4(%F2lkqR4DT0l&a#`b;#;j2Y-TdIf|N}a{f?07tRW=^xrAh!U+B8 z?5Vw?vM~gblacP!o_dPaT_(Adk^K`nI|ei9Q?tDLj6Z75@yc$WuS#&cyQ)qCacnCt zS-spafG5e=D2RMZ4P;C2J;fGKr4#-KfMwEWLN^Os)IndMU@0VmE;$2t$2O1$xzr}Z z-&n^+%Z#DVH+(QME#>Y|TskUjXLfq`8KqvD+-;gD7H1?R(DqV`xz}^fiJ?%u+fyw! z^ic?Q>sfOY+JtDSUKa-Nu%IXGcG5aE7TRPr#a-k_1PlXNX4`L$&BA0iOv5Smmp(Fj zRRUXTTe@#Ka~#oiS3;G@#xpfb%6-AC(}q@lSX%rIlNbX4*N0`fph z#lmsudcrf733k@7zn#K$dQfRLe(XRh;6sZ{YM?KuX`^YC-?X}uB1 z5SSkCsefU|jWXv9U)&r%63I75NaZ~*Ww|ry9eBMO{g5FwIm&K;+k?`UdHn&1C6utz z!@x5jGZMAX2A=XsisU&8)HPKeax?y&)69Ey^I)?mH)ZCkAwN_Frh?tV7E-2(LPy+$kzqMC2&$urQu;35chj zy;jYMHm{(|E)g}>Uk{VB^WtMP2l^JjeW!Aqkzc=p_|6c2wdae!fx{#*>g)_mr zjUZ^|Um>KCE$E=L90M&|zaH+j{1bdBM@vpnstZLRb$L{A>x_sM{Mw~K&p z0)=+@Sc?_y{yLav9by+pi}@Np=6XCIE~Gx&VlJVng=J;d{{ZNUtYyuTPO{y;6 zoGoVFhk!djcaF)E4SdnWkmz!*(Uty{l*awJ&O22lVH{HW0t`1A+Ha=vthP*7J}SJF z;S*dR;1td>icNm+xYY`=SP~0ChPa&n4GkU|+dT!Y?cze{nr@@!(&ffw`!iu5K@5>& z{L+CZJNYW&c)uBL8zY>|U$;5>wO{%^#4@?Q$L^ngxG5uG8M0gd0AE0=ES11I+#g6O zbvcvknKtcW-Jw`$gb0^Li43CMq!dKrIP>>>D>%=smlv~bP1v(D*Gg3GE6qF%I-Bq0 zPnxP99Y{ejYEkw!UZ4F#$0Mm}jR4Wq7b$5w-ySC;m%;huZb)EsWeSu4@QwO6pFzv^OcC#;;J z4Z7y@s{6ogDGJ1>5a4~NcaHDK$S~!$>%JgKQKQ(iO0|`t9QB4?YPmy=WJjek{aq;c z|9DwaZRU;rSL*T5G_-%U`MN&R_{Fa0tXL^cQns8JFn)pb6$UFK3EtVjR~u2KoeqUPmMb z@z8_ohJe@tN6V9^{Hi#hT)*Tot2Ekk5t@?pnC%c&CLUoMPJ7Z_(U$*98eGtAqap3ceSs*6>LwP!C4CjNcyy?SEY z<%Tj;?ER%TH*qf05L(~yjtZwbTyIu^eH0~k&G1JJBh|;>d#S-XK`46+geEaO(VRt{ zBxq+iNM|zh`fC6O`IU)$+8@^4a71Z= zoLpFFH}J0KPRR+_HOp}`HpLL>#=qq2S66LCrRV(0*h22-CfgF9Qu2*ckz!lcROY5m zuZLbtqoHzWuqt`ARGY?5AO;nN!3D<+hreuNCdk9KX% z6!l08Bm^cy)_T4myV-X{IWR&v^1#oJrU)?jjMv``%FtJEvC(><9zI8c3py{jOuLvr z=v}I>V3M? zOcUA)-lc>R*xi_`>HdrUgF+y2GlWz?$5%_8Sbu!+cE5n-Bl0Y=s%u~Dcn0VX^GR*8 zl0Mf$>Ij=La~QeO5+|hS=E*hbRQ7h-wGf2IcGw0-xG$Dt&fO zw~c5W`?Toage60a4|@wMKW@w_Y3^@W5c%k%H&kw3)JI&`y&kF-nUan3H?COLNEw5b zHE8J#YuEwJFBGCX)oRpB9Pid4+t1)dT~16bujST%ou9u-x#wrVJkio{y_?yHM}MaD z3!1S6l4Gq3ch1{|b7PJ<7cFs@7*V39bh2u^KZGQ!za7?n3JjDLe? zA{NcQGvfh9$Gc?RMVs%SKPSEu{oYr9bD%?O9(r^0SmfBfd)dM-+hOrR91ccb7YEg# zi_i&_AbqQ3sh4MQi9Z){7fgym4QA=vRSfjBg}ZrY;;N}j(f(eb;nALUbsH9W|z1j3ir!wUM&ru8Ap*?y}M7nz?U)>bsR4iR3Jg!KqQ&fyB~+0j6S)$-(j- zNN)H{c5d{^wemp)SqW*r5g@;gvVGKpGsUV$NqObI+M8nCIn`_7C%gA{+fkCX$zO`Z z#!dq{y~3lBFVf!=&=-8i?>HW&3{K2lN-=bvKf5ms;$iu3FdJ&&#S0ZEtmKqgwDv<_rwW<;xYDgToHS?;iz*oEga{XO~KdRDxe{7Am|0)X`a2opg3s2}xw^;j+fD-0QqkQSuc10?g z+}JZVwu!tV4N-Gc%SByVv1GyoB~&s)O^t%=s&B&-Yzw&%30Ea~|A^;bfU?9vxdR*wULDTe)@7(8mKNN%Q#K=TB_Kf%F>wm=>oFLv7j?~KUj=YxVGugA z$DsrJyXcAbisk5b?}NHxh`~|zryD|pe-&!SFhoGC^sWbfm^vQN?zzemKH*q%DdCbi5C{Eo`q)Ph8p3U zu)l4}nCykC8+(cU;fu|=%an3 zHjc;eL3Vmwc6u9AV~}X_k9Xqv<@uUcLyq*0BIpUW><_=`9PeZ-^Jpm$LJ|R zx(ChD)DsRl3ioG97J#A1jD!3<(}a+IV^ma6vJKJ9rWarX;8 zl@Qh_nLJx>sM{=(*iBWFmTRr&kzj9Ns7<#k2geD?3$i2=DV;HX|Mrkl3mbhnBSSGm;C_%A%~_U-&R{fut@Ybn$? zV{~kLw_%R*OjEM@YO`q~HRi;{Cj;hFSzGT_^aW@JhX{brCL?N1mnhsh$0E{BWtVEY zGUJ%jlj`xkE0SX3%hA?RA@?JZGevh=ClB-Buf8`|Rl6k$!JHM~f5%ue1age2SkM zaiuK6?e^p#ro$Osc6NJ<>TlOd{h`M`yDo7pC1b6`Nj%j9Gy~D_RtgoBOfBAXuKn)3 zl@E+i0X%F?y^MF3HfEkk6U*-njuO6^%00VXp6#j&vs7Nhypi>B>4M1@UCPI~UZXpw zhvXaW8*{D8IJ5VXi0dbB$pzN{IY(1!dbOzJc0Ffo`l{zqd1+Jq6kcTmGs=?Yd^zfZ zeRpa8j8$uxPdD5yPRu z_)4CU z{bw7tT7boLq`_*plzx$>AV6j69wiqKkVhNK4TJIgYs(no>U*p!J F{{qq4sYw6; diff --git a/arrow-data-source/docs/image/nativesql_arch.png b/arrow-data-source/docs/image/nativesql_arch.png deleted file mode 100644 index a8304f5af89025f715fa5f9f912544145022bcbc..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 65151 zcmdqJ2UwF^+9-+}#X?a8>4JhHO$ea~il9`f(u8mMHmEpJ+$R|hbZfoDBqN;pK`xAVOe16jX zjv0)KilKw@J<{d&+=hy3hgn_uw&4S-PunLOMDUjHXnEKH{_l_8Y1r(hKWPvzc4s1lsa(K^sS8<_fZ+{;E2mT zbt32cuBXB!$KE>A_Ee#pew0@4n(w~AnOho7S&kS@wQcwY_J6(y8@czQX>qyMgs1R~ z;)#pF2Y;ePd(HklAzvPUbUm$FEoObqBTVbAd5e7%tIJBxRjZoLsJaIOEMns`D*x3* zN6m?H+rz^tIx)_P9G=hR7OMKt^cQQ6fN{Ks-grY3EE{zI90g3yGyfX~xKYBV_AzkS z62UsG_OHJ+cMuGRKqKP@V8?i9G4|B$=_-d7lTCjD#_~7RUJRDL@K@yi9~i(p3_g2e z>SRBzS zdODCIx%;1&-hd>4-I0S+sQ3y*ZAdU4OjtTfqI6 zTReYv+M7@rEKB?6l(B4@sA|gDjdqEkjk`sBYB9qN-z{es6jne>ODnrOMRpV48>}?u zX9{k8HhJOgKza5V&o8i_9bvZwt!F6Q3&4POW~kSe3WL5?JMn$d1g%Fda*8#)o=X#g zpsIJjXKZi}XD`hSYMtygqm1B=xV-=4iPjjPg+&}XRAxg5`z9Q?gHXcH7YC90q^AsD zrtcxH%d~y(hroBv-9;0py*b0U-{WiV&= ze%#{aL=hv$8YOCEIfY{&waz7(NImoXj8pr#dVhel2>vcL)-%3l)&dGB>!#=)l$Z!h>EJm!+5L zsZ~hl(la2LKQsR7lqcynTc@H6NRpqzTXVe!C_6!|8EQUVN?WDJFClDoV^X8rc~zGA zOK(MZ@>mkR%s^+lQERyQoLWnwnlE%z1x42-0E-*rbg-C^z}>PW`ITGa>P8A+5jB-X z4JXl8%i}az*TwCiJ990UISk5zR%^`+RfkYV$#bG44tV03C1>F-3^}fN zZ_71j=Okje--~1^yAA{kG{dnP+Xo&90zO-l(f^UCfQ|U7RnsQ7@vIP};=7#VY6-Hm zt5x?XYUUF2aN&*Kl9_+{AbSn#yN8ei4glB?g!q^=LzB%N1jsmnR7ti3x3|}bXx~nj zKfyG@9-&w8aj@=06fneUe%(?pEf?V|7wP*s?%*#zF zkQwclOoA5)D#F}QOelMD9N3h@bxgqA#(U1?s6|We>tH6vcQ!c@z_yrLr_-9yaZ)z#4FUNc}D&GLkIhRR5CRx}CymcZ1Kg80i(F<1j zDMhk9^Nfd-!I}*j5Q<=UDbpb7UIKd31(DHJ{7RZ4!0%on9lh-Bw{^sLTVB6pYQ**O z8r0AZ;P_#ZWy!tHtUSy1=8c>StFmQ&FD80BReY6oCPK%rFj?^1&x*Y!9rf5w-IC5S zXL#zbJ~M*Q_G9OQe}MK`Uv9s`zwKPEYcZl|?m`8zeSr*V9B)v-C;Yb>vKK5-~%)~hj78s?$L9XBMZxH%=wez`I8zCEl?Fo=~< ztI0v0;gKxxm-bA$#SdY-H>TU8is$W+D4^{CCJV0BO9IohBpE+6UcC(6@o)LGIOeSH zb9)YErus_C<>49Srj~Z;ET!qCs~Dgi1YU`7uP-kvzH%Y~E&%f$yUKHPPcJwN|6=m{ zCs!i}oS#E;o#V#X+H$fssOb|*7N)zQ8mePZi{KkDF?W6|(dySf&6P`NTqKM;Ucu*C zNMhp~#zeKId?YrYyxv}dHTM&AOQsuwmYh_PCR*->ydKA=N5jnF{bqa0aa83%9X&R$ zy>~W0vD=|q&p6fcvGIe@!W5Gd)fDZ`_B^Kj`Rsu!?G4N>KU!_uM_nX#l@l6;crA#M zmK!n~gOip-zbI*#I99aWDY4-CL|t&Vi zmNH*qDrD9K+SAZgT6wKe?Pi=PmXWv_aVHmO@iQl|k6=brp5jmd;&Ux=HZgwU6!UqX zmHn69I>~N`W*iTexLmE5R8J=mCpc$ZXiKP9RyF-(DO3gi*sj5n^5FUx6@J@@ncR{x zTi18@E^t29i|OqJ(4I##iWZzX+XN&~Bf@^mbS@_GJ3FGyO0PixtL(<>n7{8sT2-ph(8yU_53 zWNbd;myEGbtbQJd%=#g`C_>-g5^3@gK3q_v7dr9#t(DD|EmSpcnWnU_k*Za;1cI@g2#(}H{`z1pRfFGHk#F7 zwQu17=g6%0i2;Ti@(q9d8DRD7G=lYGM_n{)Z~M+G(xNpki4d|xAN79i#Ma9PkLC9s zNcKGtz_%xC!9g>AaB(}$hIbTB0vv@n5avF`?XHGZgfz6$P?^$P>w5XqBY>W&g3dMO z@g`x5gX%u}Bp3Bs7m3tFp)KGX`#Cn0!r^zz(&-xzX*X!wn7MvEmO`GEQ@G z1>rZ7P1F>=f87}}HM+SvdS&rD>(33R(?s(S?X2p5i)qZRrAZ29tt@IT+LdJm;G>Us zfia7POh7H2wj=iyRJ@BK@Ap(rsM)2<>JoP-gdC&xrgp4pIH3z5VwRm zH0Baqf^5dYXAV!juP;i{2A_M~0S96wk?lX3feO{H@9|bwmnhlVo5jRlEUL0;)z~T~ z05ocIjIg$joY3une&4CX(!7VGPYedz!__X|ue%4(ug#?w@Kk=Fih;+lQqcH1p!erq z$P|$V`5iv>B}oSE>&5rf@Z;~ZxpjU#>%Smq?1)r0uU>D=>o5GdIU+SB zbU=ub-{)TXgUa#xnez~xVW+7tOA;zz;(VlhkmQx*wJNNkAA9>GKgI2kX`m zP-mBtrw(he+0@OzyyO{K#eRl3V^S!4YTWB0$;i`%^rw12&lccv^>?|n}osi3uarQQksWINivPQ|R_qE^J@jnALipF1@be%RX6U}KkKS2>@b ze4&m#!Y$CooL?H-lz8L#0VcL)TxEh?ZB~VHrL8~Xz_{*7=IcCDZ=^Cz49rQ7t}gu1 ze7DKy&-rfnsm7)nD=q(aU@r}B^_p=}1qg8UsUHQR~l6!ENnVE(Iug{P7!7U#6re3eSV!o5%*?fO#(0Kl$r*Y)`NC;tiJZ%xynx9La+4oqRD z@Z_4J+)_=5kM=dGUlywQD93vIj&v#aAg8*g$K%$JZfZ+PxU?a8#1>_qo~=H^GtbnQ{diS@pzI#ToBa zeBdxh_^kWp^&e-g7e3%&ts^&SZBF0mKYFhv+wRCz|9lNQI>8#OJ}Wo`_oX>&IiyXG z$M3llq7mGt1o{i>*A-rQi`(^Fdql~-xU*gBT1Th5!IhYbH6fp4j5Ib?@7UToRA_jJ zw0Db`OeMh=t8>x`1dPz)$9qY+BA{#fW2P*6fjcen14W$W7J-MLqj*gF8rk;L$JVCA z7Ia<689(ngFM>qA>iQKfv$x-*CaM1oUo`8gZc9DNDZe$f zojxtFqg3iI?_qx=^m9*7g(tEYD@M$G+?1cjQC7aMUHCTp-h9Ihc9bUejjlex9iVm{ z-8V@mdA!Tfpl_b2-EY`uMs(Ac4^vt0_BzBRe{%`*AASB(eLU~smZ!%Z^^$lOeXEjz zZ*+`ahd4wP>n7$kDfyCcM4`>&5;?MZ)X^ZDFJ|c`BY8y=tdq2>UsED0?3HVTTUk-D zY%INPZkA#X+jP@X%u+A97L$e*TS#?1cOYPY^Fg(aOijvE-CtdME4rOi`kMw{b&0`RGd9|5OHQ}N7>B7RSoUC* z8%1@N6KB@)3xCF!Ef`F_DBz@lArF|zSyWIqz?yISl2Di zfX0!hKk#bdx$@_a2j~!wZ}VgJ8)V7+HvJt?g#^e9fg|Kebcy+=|i!Rd?D zY9NLrhg2;*u*IoK_5kSrI`o-))H^!rByHS3$CJ*iJtrhgj)I#A`=QRBK!w~sP?>*= zrX=7t?HZdKS8Q^U6zJx(@S~iT9sxsrC;!3L|MQn;!d_y@vg?E@GsBv2lv)1!Z`!m= ztDAEvHvueRR%&>!Kha2uxc;?+r<~I-4g`>OJ!GNvAkD>GE=MKT!LxUW05X zcy>SPYgsJ1u{2){aHXUOjJx(D$33sQxx-L_Ny@?!TEW z__v+^i(h=erWT;n^oOYL&s|E+F#NxlC8+T@R4Tf!aZo{_!M@vD1tz6{;ZRo;B#O@q5b&G!5J&+6Tw7^S_NRBTCmY3VoA z+eNJs15}~!r*CeUf0CgTp~!*&-cJnQ{7!WHe&ek&HQR44F7GiH_g|&kU-ri%3P))m zt4ZZs%r_V)^&xWV)N~)1)9H_C0GC|3_6mnoFDv$l`iNU*jqd)P$-mCwJ~t-)HQT2( z+i&(8Iop<ejQDQ}1EW8U0sK?J`|b&2@hCkA%6&G${z1j6hDmGfyl1vQ;NmvvA%( z+76nZ6bbWMV7<6snxUe)#kp5%cy=5oH*u+}KhXCk>-2V5xjXpH@hb9rQ} z3X%&2*QCo!uJ)Oo*CYoYm--xPtWBWmdY_HN{H#a|L7V00gJ0jm5$~Kb2#D8A@R%#n z-I=s4w9qd8&c#pk^rGi(TAaCVF}S&I<{?Mp%H_(rmq`3e?YT$p7g5Lv%Xgn>p@lPqq2a|%Pz8KY>N>PLC(y0x-oz{gjBw&mf9OTS5o$**Q{0y zF>W!(>`pkdPt4H$iqiVa6uUIVIY(ALak_3Ume5kfz4JXUd!ZpK99``x)j(@l>fri0 zeQ9O1XKgtxZMrOHs4E*zaS?(OD%Y9`XR74z)hM=``X{u_|LBO*aGjtML|~zPV^uqY zbmU=-dk80l*N*AAC275P?mbl~0*R9?{>mTXaP!zd3S}}G+_c`SOHxt2H02mc4;Z`g z<#~+4*N=g?qNWYLrz81;^ScE;6YERXLl`69cYZf9zU5e2E1!-`|EY z)CPQ7D)v7AVvP@;5s!%*Uq^r%d-Fjo>hp6mTNP!VER_r5t(yNJ(79w*>9_gx`Z*os zOLO4H;$kU&db%f1?%N*00d}No?KY2!rZy+L zBx&I1FOQ^j#r58^w0j}C=5-P4F0pSXn-Q7O$FQe+&R=}2(?NRp^19}{4= zxoI`{2%9R{7?lgWjg-yr7zT~h3ehVNDLbAbsDK)?kZ{) z>0!_F%b3WAy`C=yGUu;Csg`6AGe66BM>Mm{!i1yE|5#e|{y-!dnVlLUB^tJlpIzY{ zf9@5ToESKO#Wz@$mGWv4>`iJ6KKNP?R~TLT^-(jcOEQ5Y55LF?XfD4c4N3KZiH#b1 zF~wqe50L1k`0|LwZYUBR#<(NVI~6W*1i(CE+f|pr0E&7ZHrHG@b1sD8rwZIll?zwOmJT z@eaNOr~CfBu|cb+R5J*28o1D8oPY?$Xc9DN5)_~~vUFFv_iz=%!mAjnmlB?~UGXcSA&dMDxWdYNfXxy53+a*cFh%y>Dh@Q+H# zp}U7@Z7VV=xae`UGU zL|Nf#EsS2F^IO>`I<{$A1^SF%dih6~_>D>$dP&{E35lx3r=T{f>F&z{aV!}dKw&#p z!ST^yX-klxmU;#u2NvE`U9$=_ZdSeS6L5~7ATEq2j+7a77j5j#GSi-m;Q>i(R4ZGqo$Nn<*utlK=+2t`w*~*gwzbyTV@=0gYG@K z@ZAH{e4;1RR}-(|V(_3OF;TtV9`0`D!$ju)6n&ft8J!UG0FM`|-@6lSeGGiXZ9RiD zi14c&SVDfq>ZC;~;J)S%=mSTTtOb<#)9-3$C2a!s`mwoBi~mdk;7VYL4%TQp2I1&( zzNJsixR~6HnDjaN6aFD5-jwzla*oUo{Lv zysj#XZ>THVPFdYrY$$1SuBSkU3-5p4;#{TXdwo7#w9G;J7P@K@SR&_oO1QsNi-|Pv za_%>xaF2lk5%40(GZk3kkZxZhEEFgvt63z=A8V~sX>y9z%~ zik>g-wNDe67ziHh^i@OQt{9KZ{UpmDgotdfKuBIM=q|4CkJf-yT*5U<5;bzlo;OFV zZeifjORQbjr@IN`qv%$E^x9@IH+xK@4kJ-e4m%7YnM(_V&?c-(2|Z0~$HICXawZTotoC02`F%`GftkD8BH+^>50~&ftsm^Sw=Ac>g@g?%kD0vq=o{AFN%61X8$7LB zx~N5pvj^u^hZbcDY;1vku56>Opgo7bS=3|Cc%;V}#zdu2AMzj31HEP!4b(9PcS2E! zXU<|7UdC9wBi4%UO$$OD0!-VZ|S+wPn za6brM9JA`5BgdXtq^Zw2TsN5sva9YYI@EtvoFH*d@-_Azfr@uOJ7j5HB3P6S_~t z1TFcjOwf?K3?7q_5v(<6dX*Dn=kPrA14mcoxbctIb6$$V4#+w|t5-r(EgEmxUozO? z7Hb!K&4unCxH8w-HE+D|WTAQ{hLcO91>RkAG|)>;&yj@G&Pb)zra0r0%r6~9j|pqQ zq_swMUm-uluNL*TbpE#Akw`^VLTT9iix=Pctn_4h`ifAlM4h;<|B#RgvhalsB<->3 z`5z@w+zNMcf1-~}+U6(CIj1#_8)ni?if$GlIKw7r_ACE?-xUhHdl35gN*`~}6A5}G z+}?BbMgnW#>Fkw6Eu1^d;@189V9K|aVWcE11o(I&hv?#H@YLe_I{9B=g5huU1^TXJIi z=Gl!yLQH8^eZ<|eP^{3wVNa#>;gB0fiPb;;Ar{mzT}i|7^32a~)vcp{jZ~lT{HXg< z!B4rpUhn_p{9otX{RnCqO(hTfwRC!0@gG7`O5|?*f*hLHNexoHeZA1X_4{I>*n_CQ zj&mi52->wGQ9|;!y40q}-8PHv7Va#b-mNmw0m9Hc0YvXf4QGi zII?`ZnCIk=O1B>|i^**tz8Jb~E%sOBb3Rfcs1Q^M^GiFOK_y)YoaCJkln+335A*vc zR8;?qJ<$80PzL*NTdNP@;V(VXhfaS9#{qP=tbcVe?q9fs8N~kmHiGP(|1b3JPl39GmKd-hys~~CO#~1cC?H-HcXqUllo|_)ryPuc) zHZ6jj<5eKf6MNeFg7e@0;wLJ16eT|t_Ho< z(Oqe#)O%UKxaB1nGpJfGg(_;4ywrp8QS#V=#iTPQMb`p#=H8ENd{+A`xG|B4b4n0_ zou3KN<#;#qlT(MSTHnlewr881EozZriOzPp(t5@{pwY?eycpH*|FZM`2GHP|cr%;R zfE&<}&v|iPilxUQtRYi7)ar9W%ZuBy2L3v24o!$`mtzOX-upM$G^+u2N}^@2>7uHF zmbb4A+LsT<8uN+*@75@;V~%H9RDMic>j4N46$heEN?heMMq}%u>fiZUw++Nv372SH zxwL$|7%!S-3`+7%{b*ehas)Y%@#FSi29FxBmu~bqy zM!I?Y7kf>yX_~v?=7KuJhzKF}OJl*pMjRo28T4`bdTP=o8d9}Wg5}2aicz@QWsc*G z8KPFfjFfd;) z;XB^Tbc&5Ue4w#<$01&AFdec%9N96&7P02zIsd$JKvEhlhf6JUITJ~+FEIFw*X246 z)MM?tL4-4UWSqnQ6q^R^Z%vqqh5`l0;SR97Rusf${@s_mmO#a&rw3KmS9$Slv8QNrDd zteao!BPw%e6vR}Y%Up^QPmG?tBPK;E=PR=t!(9MP z_-F8fTICfTyq3MRR~r{(m_OWp%LeIYB)wwOn~L9(%c>8T|Fk9_v2@PYGic5ub|?vY zG#e;4J~Et~&6V%1oCc3xvp{>RTA10k;`9UW&Muw7GyS4YlSES8yL6BeW0JQQ8xu`Sv8O=Q`+H2x-k zP^)j#^8q`k?&i+1HoDUZ#gZ8|_tiDZrDsQCF5?8Eyq~vujze`h^SIu6Wk!X2vZzCH zxDw*-!souKypj@J3r@wJYHWyapotnLxAX=K^y5TgR)4hI&9N}a%v~N2#kyac17>rv zz-6CEg)ixqEiw>mp#DR(zx3@RTg>3wv}$5eGqAPfhB1qfe0fCzBU>>^LkUpMk4)47 zyN0~u?YLyzR9o1~=^5%7kWba@o_R@@~8h>gx3`fXr1m9IS|!+`5=|J)@F{ zr~(XG-bmG*fWL#j$Oe`U7x<%H;&Ix`J^`F${rR!Ufer%~F`7GB`jD>iyR;WBe*E)z zb?=W$?{c;C9bV@+2`fSyIAJMdG|M{{X(~VTZ1l)T_U%8=oYs3VYeTPf-dGbWwh-rd z&Se*hoO3sLsKOTJBD5CjY05u4qmdI3ALV8};9J`nmVJ( zq_gbuuX6t)cPhJP^)ZGWD(~!! za(5?Ene!6sw6ggGYo`;kfO*c*nP;G8QI=2QuB|1TpWk^83bh3Sg0J#oM;Zi7JTPon zaL+n8amePQ6yv6$9mCVgD@s)szFhdcUM4I!B{oVp!s$ZecNOIcLb>_5re3L-HRDFn z*1U!BNR|0{j0JyUM^4#y&vr&w+{D{w3)e+p8NUF3Pi~Dio^x)D)Uh=~v7nC(7}rIK zX}6ap=3Ab@f?Yz@QIB0a?=xLwy+Xml_?5=*{NuRdtMa`vq{n5q9UPBq%M}GmE?4Dh@JfkQp!sO|3*@K?4@WvupjPfS+gPbovF4Y%m zW;TSnB9xo+fVk*JErGC3Rz+=yeErR*yC_C04=a&&mQTMS8~ zY(6Gxl^=a6AUPu%vLLZzF>bM(( zi%V#4e-npoP^n?R?Xm*y4e|YmGx&IKY25aSHZ;sEx!fT^k3|@Exl;wS#8G^{k%A^R&wQbLr2i=s zUq+K}FpGKUG)##w$NGr>@Al69@(U^D46;S;kc6LNs*$`fj&+Uf&o>uj7X{vFA7&i; z>gh>Ro64?Q_R!H5>Y*c6XptH2`KQ<@H{Mm`{GgOo@X5>2ToF@AhPGtV57H<=Dln2#1ri( zePiv1&U!$Wy;fm5e__j=;-=KWIX~EDoY+R&F!8#RbMj`@T1)*%fWUBnlEpBD}GX1(*F*DRZ%JbOX_CLx)FHz)@ z-&nA5uf|rgO=)}|fA)@%?9Mg^wa1HBZBI3~miD&B>z=7c&QE{F zt0k3{PU!5{4#|()*CM;m&X0{f97CL)L1zXX?=43}`Zid#)8R>d3Ppcz3#|~`bF;N* z{&dE(Eh*mfe|&+wV}tCvj=EEIu@TTuxpz~_!%nUxW<8ABtN;DW;_Q9kdR&@}PsH7y z>v$ud?e{~)OKodw#T)(|c1B;mFWxPC_YHaX$VUVQfx*G2S4@g-1ig1N;n_V6a_XXN zy&_{xbgyppYoL%YtFIfynyot$r5|W2L|B8CvkJK%?3j#-9|y3?e=l#VXP2zPz2tCuJN{j#%mE3DeVEO3v8#ULN$zw4= z)6*sIc8s!Y+{TX>t3Z~Lf_CuBqp**EeEl3mYODC2Gs!xmxSnTIVdpk{0k)G~di*~7 zyC0wW%jAEymsb&7*x7O`T;JYQJ%S1SamO^Ia);KP64br4`N((8!}Z2J@hEXxrCjm1 z&|{_CCrI}{VyR3+p(bulQOUHse-w-Yj~L%R9q8k~J6%O6JB`p|I~A=L;PC}B#XHJP zYYaZ0&$ba);TpOn_HwUGosj)>@>OHsNB-1Oy89xMZ&|jR^SD1gh<$SLW3!+6X&;0N z+u3OK0NF(hh?OlDAZgiVNKYQd-{BZ!@L(8|fTdmMuIx9-md;QrNX5q< zqtAR`t?|~1MeHHE+Kwil#Hd2bxWcZ+*SD((AIgHg3cWUPWjhv1J3fnNID#t2fG8 zO$iEY^C@2JxW-jVZ<1`42(3@4hP8~$vxp)wC6N?+V?4xLX$mi)3t5UlJf;U$axzt}q>#{o`WV|;%tfsu*BE{rQVjL%8(;|^O3T1v z@B^fAE!Ys1k$m`d0wL2m~5^m?8)4eHVunlVQR{enaqx{WC%|+ zc#I|=md1wDX^nhv@m3oQmYJT(#5R?=*oC~!_%eHykV&@y3(yLwO4Rf{S3Z6*B)dQo z=#xg*+F0F3Am_D7pv_~CV>)WO(x*g-Y`qQZl%nhhUJG{0G27kYNNBLf34@o>9j$wv zBV-Nw2?2rcRc7=cTJ#Eb zl0t(JDoC-L57vW*gJ9c=q(VzsXWV~A8M~&`YOraedy0|Yc~z%ZogQZ148m*S7Pe@YK2^4zRntrZ5`j~3_vtRy>Cr@IiaeV0-6vRx-rwu43ni$b}rT8 zMdQY?XCrJ0o~AqTyxnjY>i-O1=L;pTYcjVnz>HN*_rZ8Zn8d8gv}64z+NGZ=ksyd8 zrK-^${2da;gBksN}^)Q~;mGRvM?4pxMp2)l5^|%tz-uo(CetpNaDs7!ZWg z()8DT5bOm2<7ULgi%3W%7lFtIC%!U04Kw``;dyy^Qn2ej27R3E>@^0~4_GS7F+`IV za0#J=`puQ=+g?-4FIyz>xx@Bzw`n{(zqM3OyNwsoC^#}%{9LHnPxA*mtnPv{AzIUb zLK{E9HbV~UI4dvlx9PX@D}d8=6@3KVZB*ASL~)DDBj)X8jG20}_@z8g5m5*((zzRJ zDr)i#m*GigYvK3kq6fj9v5by#)NQDtQfik8CO3S5LHjOAsJtiByfi`HxsLp=9)L+U z`JwCO6JYYjVqoT_lv-#DLRhffMprN+uL@rB6Md0oL4_64$8K{Kfwsiefv6Ju5VIyA#_8|t4_5} z(niVRb}D$DqcC#P-zsj%jdqXGnnPzXA(+_24h;vyLR7hDEqoS^C3v&$Ef*nJX>aw| z_fL_1@SBfYgJ_;StfBeZsQ0AICrYE(6j&z+N9o09gS1c_q&@Wo&4dpSq`BrgO$qKy z1eI&m@lmvStNA8fK*h@`=CW4h6ktU0?4?m!%pschLY8LcwPa~J4FW%3C4rIy@v)N) z_CPz>y_Bl0yCFYAo`km$*)oKe(m~kD(NJNCrF~5J?wPoCzX5DuS6xbV6HR5GrYfZI z=UK(X1rpDRtSwe~x82ChhWsXZ;$%_PYS8*C*7L+~l8dJ;39)nDXim>EZwetRxHyAR|)~i5$@R<KciQ2(f={=<_yZ57GXr`#4dx!zd5dHi-EHq1oniXY-$s2Y;*ANJfuXE=Y{Ci8g*mf`-=hf5 zj~Xkql7U=~=B{qD1C8_UmR({dzHv8)L}qlpQ0mR!SBc=oTY`y zUuSIVzO)T$U9wxe4*jW$>IETFw5?w-NJ#hI-=;S(!g%JL+iYn%Lu`;i(H!_SRCM%& zGt%0!G!mlQWfx`n6futyJMvTH_OUOw45?OTR08lZl$)TOl!cHm7sJqK2tBRJnb z)j3Zwq^l1Emjs`}OlK4RxGa%lm?9sO7QRxlx_(`YDf-Ka{_>umO}4-*!z`9g8E$#z zZ_}ge!p67U&QV;uA!i!Y^;`|k>-Sy}kLg*->cCc(d1h3Ni6Ei4lfM+9&!fRt2xOsbbgYN)n!fNo zKCMmbPN&9GlEz^|fpO|Zl9tB#qC!^?BQ3wz%auz5>oP>my&lvJgO)^YwfMggi;%YnlP#`H?7+$?VhQMy0y!Kw?k+(W8pb$|rVCru z1da}W`?>Q*CV53D0WEho3>*J}MOABTj^EJRU^D(9Puac%x_C+1jg)h+qIs>h4RBfP zoypt96&9)qlm0JCoynbizPDhA7%3JOk`sRAEKxRfG1f52s(DjzuH!_MBxAW}^}wfC z&lFh4zOOoV{+XnBt)dY>wDGJ|otyx@Zb~Zd1TsY^7vBjr%s1|DYPe8m&|T~#0p_wx zs4smNFFIE;__1Y}EW1#fWV{AmSJ?j?hhbefRY~~VfzGeBqdQuFG z(x~LV>QXKFiWRP5=2t4TsJ+~&luj<@D``29uD4H7bElI%Xq5(_vC3Rd;*XL`+}&$! zu2F+E_$R*HOlU^on%@9nNZ;VZE=>oj#4cHpRrSV6wh8h-Q?Uxh)FS9IoTTNL`Mns3 zKXOlI1WBtH=41OEG+0P+hE{FxJ)2)2CyK9Mj-n!h!epDfDY6CUN?LQ0El=`d7bD6U zgfn3%goOu>Oaox5o zl~05G>}eNjC5~ylP8^{I4J~426%Z6HLJq`As7YyCVjI;~T}n;iVV?mfMzYWj<^6u> zC`bzZSc%EK^YagXdx0Z`d$zn{bG%OB!7M!3&ru75as|F&VVu{?$4f_`W10j+^ly~1Gdz`gtk zh))24%Np*UqDsPw;J_0a2TFhG7SA}5zGFCUH9B1Bnq<7I^=fpqtBrE7OKO99>Y{f4 zpCmZ&o)Pfu{y!ebB(LAg3V?FlEYaNJJpJu93biR=WLw&P>4kBof7UZR(XJvtV zBe%OotmX^2u7-?fd#-H?S#)V%yXOH(I!8oa)9}mS*-BkBzXl;IxmUHY0Fhyn_`${p zsMG%}7Og)wWw#J0{Q2pnDYed<(rf(6b58f|#Sfi_rVU4qqRMWz)r z(PS6r)0RxI!>?fJ6T=%DYwU&Sb17R(-xQ2moi##zQ#L|fgxcuYV=AND6ThXy9k5L! zuC<^Kig>a5fOD`qt!m8!)mY7dO&f>@G_IR;Rkq){))>d{dZ#B%NB!Chu#nKx7$j&R zIHp>!$ggUFz1Vcd^iGu4HJ>NxCz)hp;5+(k0Uj}%P0uZ{OxNbXF>?_0&8mnZAYU}j z1iE-##eX(RQgPGB+k%xC6^3YN)h3muRg!(poTu=Nnbf1#jT>t0(_g->={uhG-v`Id z)ZH>78}a9Mw>$d_`)+M>zSwEnz7OTHH5eSgU!u1rImo*TUC|n9nYuOYhlk=qBRGm} z&!Y#%ZCaA%W9y?(90dw?E4Ajw4;+Y+C(;tCb0BFZ6}CF?DJ)e2d(sePchQpMn0>VW zCJ377qqj|V8UPM)3;awE^isf|IxCxLpyu%NSfkV?QSdX>8*7cS*fscEi9{Y$M#JM~ z^V@nMNVHNkE9*Pu_3Sc(ot#^*NC~SlW;Ky!0;Sl&X@Jj&iLACcJ8$o<{y>*Tx&OtH zlpfs-ah*#UQHvue#M1c`dCr#wlMgx_##Q^9<;O zriMhF90L}nd%lU>;Y#w;vz$==Q=Bd!QuP_dpww5w!p4|Y2XiEO={nQ%d(AmaveVV> z*>GKj;F?zR?1K`9jXRrROk)6AVx3B>e+DxT(bn@u+AkSV=1Ice#ovfQPnNqBqjI%E z1g*Nm`{vw|fi~$FR#xz;8aE^8f%193jILm-dATA3RYz%?5q>1o|3lfk2Qt0*|KmDN zH<3Cj$*puE5ppM&Q$o3V+0N#ff zwClVHOxjzOV5)}7vFB}vi*9&4kK!t?xDsm&HO(TY4Bd3C!$GjO`5*lrHfz<$vI^Y3hDy z$7&3I$6;)=I}?W)Yf3j>f+O$e2>#s=2=`vt!IiVWYEr)2J>g+{Gm8Df-8tj>yWSrz zgna@PI0`Xr%@4pnCzThh-jFb$YMb#l!lM_CIVM68H$$0hFV#j zMIPm8mnIN?2>0V2p00-Ado8i|VZ_C8Y**V_XKP04aOA)Tyb?>yfEpL*R$2~`R+Y=(8a<5d1C2JwCU0xX87L%#c zOF@Sw4g*Y6c@M(nc{PNO9ZTrpwX=LY&j$}!QwD(|3+N+#CO}x!5DX(pb|7LG@>`Fi z)dpsZ`J#>s&m&T#Wz`-aDm7-WVS-F^S64TV-9sr5e(Q3I=rrZK`pQD_r`mt9^@vTvpmArn&GtX8G z3ly-dj@)ULxuxvQuDrTB=i6Q;9hV%Ip8?P?>uLm}?GM}H#hd4=(<+q}(wd0eTp6$a z|G==kCY-62eR@EQUQncg*&))KdOljqeuA;O#c7C zZdW$L2Py;=Su}F4fMYdMj@~{t=fb!v9j5Ogh*!EB`<5MQ_BORnvV?Fyx^1-1l`Nq%i|9V7dvz{W#m>``dpbNfyZgx*8P&QzFv^)F zx(aln*M5@XlpT?s!+7}4Qd&j}peH+zD5m$86>d+MlUso*A;bi~{r$I;)#tYTytZh; zPx&MMG@fXGvlCS~kHzIIpC|n?Yin@|r~*d(t%S3r@^u^idan$u9tbo=2dS^q7Ef<~ zY-QZb5V(Jc&|6j?gT)Q5wQ?GumcEt5F#E&{q?ya+iMI^4xRt1EW;xh7Cj4ZFgoF=QKE z0X$gl(%yryGwY*&FKu2}>!U!_H`d7XmF;bRsEDiQOUW}bXRgxmGNtZcD3Xm`tt)O1 zWZ&Euw+^4z6^lM-B4UQ9?2owGd9fs;#X8Y=Lh1md#e%5zTeuBBwAk)%@cswikFogd zcN#uf&`G~`Q{2gwtF;ZQLH1z5y3IxM+h(bh8V<0?8xxhaKU)XbTusCvU^j&v$ zslhBW%(5(upCxKg@~ZQ;17GRF^f?)zbf0A0brD%-b(5a@04TLKm{Qd$p8kV>N^&|f z)VEim(2R1t#U_fJCkWK`;^Vd*5jdh->2mW@X2Ov7TijtO;gXNx#wNB@`QB{kRzArn zJ77e;D7J}V9*`G^S;6do*a4KMOEPTruK-2mlbsE5e9XO|-%XWao~)r@1BHq!7qbNK z%x9}Lgq$*_zG3AwQh)}5_`wBk?YP501NuR=g$a_`Y_%Bhg^{UKX0LpJ(lYf+2qMyv189Aus*c-Zu|?wgwy5TEgvM z1j5?oI)9+Q?ta$aN{B^w)yjGvHZvkA$S#4ccMMY_oVMEPO~kA*sSFq{VeyW1sR(`_ z*W4tdlr83SoJe}D7U0&nUX7dd0d(C6btx#szwdpm5htHI?bRB@oeDImy!k$s{gYUM z_lGZKUX6WWI_bWMq4Q;p2K>@Avl>+#Ko;D8HM8aj>NYSVqx4K4WZxzqPpS9hz8f5i z$;5HL5@d91lLv@{eO(fzYF-OtBKa$SBDsOCkU3`K#=llk3bi)htbcq#VoFygIoD^~ z8$q;nD@6tF3L2-hEL4%_kWfQaFZ|{n-IPvdt-{@a9eqdl7MfS^+a%!9%Je@6q9&fJ zSYih5rd1MXkgZmG{>F6(E@<0?bh~Z4B48UM7H48X_-*f>e)RC`?bWhATofD?XrOG9 zF|?)%`o~fdh$g5GjHO^D)tVbf1#l-yO>gBPun0gNzHVWpj8#VWrbT_!MRje#iP=zv zYzV}M+PJX9{tDn+S;Kk-d|fS@vv5hrRD?y3*S?o6Hu|w<-(aKdipmCnSugP#E4eDjKW5yY(Gi}Rh_HRGzo}EwEYuRX6Ap`vb&!DI?;4rysP>D#da zrcDOlM$lsHL7dZ6I+>1zLMF?&*S`Xbv_uVfQI;&J68)(sB`&ip3jD37!fC>LDGjHu zU}+e%JsSt+RM~Jxj0LPoCZa1j;bxZuzFi$a2?T!4QQqlOfS8k@YZ_MH1&vgmRDCk~(* zXE>1}54ajtClRDl96vtWrH;H2Yi+;WF3f zKH*aD_47t7+1eW^So&!ps9Tn%{L@<&$vheAP-^{lj`={TL=b+(h&JH`LQGoHt+k=|_NN=~s4a{DptIft&G$m9|o- zl9PdItz#%)Gv#N=z)Fi$)-fAI0QS4WowHt##7==G16G1?5T8lY=7lzy;)5%S1Hqk6 z=ndZ3mSTT=c2j-BO3PMp<4Oy%VRfjvZnh7a&h~%Mrd~95dy$v(_$NGMbDCxza9yyD z)KAxbJApARov0@*&u)nt022aJ4YXsv%Q;nP6F{r-L#6nCm;jF@BFR`VXsvw5V61t7 z$T?v==^3-`tL}&WGh$f<-N5)_NskM5QLi4FhD>@=d&FQe|CgW`%BEE+tf;=3^yq>i z0)X$F#Tlox<~QtmyVXpEY)_EM*xfH5bH2sS)~*oM{gPn9wM-*<`yU2Dldq8;w=gg| zm=};;--QCmGD3!L0s{^c27cECw??~YL)lWU*tJK1aVm(Cu>;EPERB2D1%C9F0gJUt zfnZl4YLLl>?G#I-_(5kca2rh zOi)5b$f|yIr1pjSh%=5K?|^vph#|MCD!e6knrmN}LpHOfJAiq!ho!$~l=MG_36bDL ziJ68xk`brs_NSezPm3FNYIu?3=z!9l5*&#})dqCy_WX)sur9(}gZ>70gk`vcrxNJS zc{us^X0S-jIT!A=XFgq5cFwFJizAkmzr87J?UD$gG6bp_=cIlT8es}p!a?qA?P*#k zxn|rI<=XU#OirgQs2in&-;$3EM=>@s$u;I~#jYye2*mrV!EP&*u|f`uW)>+(<`ppu zGnP?RAn^F-Hbo#o7X``L+rG1;PED7pxl`%`IG-AXPy1~7;okdXTdEG0&B)hUIel-O z+Q%2viOt`g4godqs52hMn7>2s48e;BR2Q;MLJK*FhQVNoXif^&+da|`O}(}WslVXY z^aXo+X(t$qLl?QWE=PxTE`|o|nO>|U;{|!dx&QTY9itj2m%?yGkU09xPdv3y=;fSC znQUhg>$26}4UO+>^M7<^L>l*+rj;{27$Gypxbjh~Thk1UCcp*_xR==tV;Mb}wKW4p zW%-fO<8_*WBlQE-2MoZ8(@9rFgBjUi1B>+$V5LRuPMRl_Gx)XtOnMaAHv=tlj{D%G zVo_YhskM~0eC3uWyipEY#W44k$`-&FAJ5LF>p^VF*z)VwchJSa6ZiT!Uw!jltn3YoiFN7QCaWH44mHF&`qpS=5S{+IH z8kYG0Y&5f=q$cC99z!5Z)sJIK<47*v4tTyUO;>u-Qm>A1)*poKd4BZa8w7 zWw?{4Sg6o+B$XqWx>oq(wc0Yn-ns`Zx%*iG8_ZqzP#n>UD`G+us#Q-pC;e|DsPB&; zwbj?Zk!J-%avPS@X*fIBl$-n@XWWD@(G<5-I}K80_!_Hz{iuP%t>7;aC`YUJ-5>8k z+J&ki?JKQj*|UW`)xgUq6s1-Lwg(9{5+=Zy!!-p0DBG@Gl_om3STkulUvkqzo5?t2r+Y&VrU{jV%=Jgw;)Y;vUP@>_jy zwz2BQ$IvbNdpO}GDmb&2+h;$V#XSxx82sI1Ke8YUWF8KUbo$Nlw(q|uJXi9>Xh@6Tm2mX zqB2E3G$bf5zvX=KBw@Lwr{>#B{>bS@;RsCPdRw;Q>~6@t7G+qyJ-Q~+;U;t>&-s+O zZX(HK$gV%~8Tn}$-QtAZY8b}7yv%JKWi`1T7{f-SA0=cFj3ad4$DJ{(M_0uyk3)kl z1_is-(=3@%ES9jqMYMO(5?(myrqrk$JjsKL0XG#leENbk0Muh5WSdG&)}T!QZeY9` zT?7QY3F^OrzH`1PS2QLg1$awOKNcxLW{SlgzTg~zJpwFgMes6{Wc|Bmw2oV<^&aLa zQ?LM{sQ21}d5zae<5d`Oj*s9b8D~IbYc5ki7Y% z4^`R-o!UGztfK>6-feg(V4r1ol)Zod{PnS4VGZFzM&VJNE%8?x-7cEfFPwlGEETuY zjA`Id&#dh!&y{ATe1U{#LRV$vpOyP>(`Ari5{+dIn*|FOcP{#rKqcZ2| z-2hyGL@?HpzECVSwOd2(ZyvBs+EWf;-q&dbvaXg+LL}^e5w%mAbpbDt;C@SlH+4{h zb0zzhanNG9fe{|+(N=eP7;p_rxN>KPjXLr0WbgtN2Pw%6<8LcX=V(AN1uftq@cnS2 zYr4tO>yz0M!N;xEaKa_)MUx7js&v-gTtv)Gw0NdE$g-Sod27;JJe$jJ){&m?*?u;u zXw-fA;L<3VU(oiF>vXh=Xg0xTOD=+ao^JN|o7w4)BDlO7?gOq|FyPY5Rp@R*On~Pm zFd*eX&Ik=2!HiTX*I8~7++PrS$IKc)_2`*1PD$T_>DyCg$u$*1Y8Tb5S%XF_gVtcU zS%XZ6f|}zE^i}oZW%b|(akPZtY;#D$P{;$x9l*Bb--a?p>N~xbEpwh`N~RoSp)jF^ zw)=H0-gQmP9k{jcnbqPG`KBj6EkC;LFko8R@3&aqwQ%sw0?IE0f#>K8=$~@$Fa@%} zY2$gGub(2+??I>x${75Vceo}XQ4)Zu_h6d(g2PE#rrbw+6?iHo)b#3RBHmqB(v#(R zc<>uX=gcEEN@eU3!-3Lg9F&^ZlDrxDpaH|G>vZ)FR*2K6)?e%!M{;SMewetTn!9z z)I!hC_|y|QN3NwuO{z)?;AVeeT-WRXy(lzJHxa!noysv zQ%mfRW6sx8%x(_MTQWhbN z87Ms6coUS&w(|y0TnR=XW@EaFIYTz$$qmlvVpQ{@TnscFh?uRl2TELvVe4l1UV(k8ucBL?&VaS{xHAjUchnc4eBM5|4w(}hxY!j_wQ?QLW!yXP5g*Y*#- zn1SXF;BMT6?|h|`yh)Y6Jg@wZ^ujfrH34A&35nL>OGTV_bsn_28SXiHEwhJfHvhf5 zi(^ET`Y0D-#s9P5uScA|6Dr28f0=Z3p6|i#WV2eV@o4pOOOaY@ro5s`Z ziI)5wS25-3AZ~U5R0@)SdcR)*((G^P05H>B!}yJgp=C}v^s@zdPY2?708BCnhM?f= z`2x$&Al|BcV%XB$?TWh{(|gB4wt|v4B>l$3uGl}?v_&yUr(3hlk=^VxWmCN73whHOvf^`E!5j3>~bjup8cpRl#$q*kQ<2k1(;r@;1g2ESc zK`91bC>HaAI*u5o#wf}dd)*q2un=JB-2=`;KhX)FR@;D0)stGxsfxC~LI~L`@ zO?m#?-?(7GA0I^&FQ5lo%0<*aPBx~j6OdD;s*HC}t7y}^e;^hkD`AAQ04*|1>~JoF ze-{#f772I!Kj(1SIZHoqhTq+-fTl-nX+fLKWQj#FA4VmvX*oNRmyxn0a$Qh+?Uda) zjN#T$aqo}`FdZ3Mw_n&UL5XmptBQ_?Fr`$#Sog6B?THl(}L9fd&Cy-N|x$V zeq)-2xXy*Amm|{k33}fVyH}9~vj0#2s&wKLU5>Ph&vvU@Hz)>_hcp~d*mbmO8iHKd6TNQ23-3qZPhMw>~8j6L647Z4`w+48Rv^I*A|2JA;y z09N4tj5h#!)ItJ4i0Nj*3Ab;DC%qO`am}^-461!;Y%@Y~iIp~8n@xV~+O(WYZfWc8 zV_MxK$dYu939_mE#9F>M+;e+dUEL~H_mP*rk3k61{q274#kl?i zIfO=~SZXCj%6_GOzJ#2dfg-qlv&;}g7dyvU-cj}K`qXSged&V-kkr~q&!u2YG5wuL z)~N1QL37<)kUbo5&0)rXi*ltdigwn`hpwjB{?eO9NK82sy}_#CZ;*ey@u=En6Nx}G z0|R#uJ--RzMXJIE<-isuODkuMLL{K@wae?RMr$4O<$3mCPPL8C$$Yib-qx^X3Bt@v zX1b>@e@8~&G4t^Fjs6rY>%&cEJi*6arC+4*iGi36N+(d~b=QLO0EvU=ne%@2N$8Nc@zNs;68Tch- zFZIRIN1))+;$uySbE}Oz@<8$I~hw0cTw->)FLq#4T}mNB*%g56(Nx;)2D)de1UW zm`l*Jn&#KFIdyYAg3-=|->grr1~{7%Fc|9LQvaQg2bWbmEVmjcu^Giz-4w?%aZZE< zbc?$>?r_i|Q2vR9= zShtohVTgL@diL$gZ!D0WmGWrag>E2M|6ccCsl2eB19{c#3k$hVLEi+DWI+Zqx3abM z44Mh8%`~2LpWIqESbmIZB!_$$=T8(yvX44Ts-rD0^U^;aRKGJBNS%>1*bCu+c!uvb z9CPbi&<0WUUjn&CHc0PS<$&7OGpkE*V_}!nJhqe*-UP=&oYf@*zlv*Op+=Zar_W%5 zLnx*@yomjtXcx9OX1iEpKF_sUA>(zv5)yA~o&;TrlwHmg(gThxB0kX@V(N=!qvl=< z2F|{3SfV%H-Wl0)Lt=L-ru|95hpuBqTf3woNgTX;F818xk-{Ig6bkGT&;cq?3&@4I zEYr!rVJa?^Q`G_}ixra#ke254Ft)1a>scB7C{*1dKaQ1C#U$t>tKe&t?=(=*i&I$H zFL~@b9{ky&y^&s+)Na@c#`$NTt5IIr8!XFNxNyH%5&7S$!DvKOrSbBo0O z+dK5AB+dH+L6Z^Q!e0xqAZKr4EkD?@LiZSOmb{wRz*xl_0a4*{Pr&f0vybO4*uRP_ z0Y-yP0`S7I>kjeE3csO&DlQxiNYBv}pObBwm-}?ORB022g?&hn=IJygy~m$;4%o}z z3KfQ%u(cZ>K&|$CNjff8DUWVS$|s&lOuahGom0zVs^hibhDYbr|H0wh7TUVRc(#vlu1Lt zewiw%OrM`vL+;KEdfe)dI7T_b+{zvyV?v+#2_ddr0m ztHq2)C}1dKovgW1hr$6H8=ozBE*VbVUC;Ga)ErZm6z!*mg?Qoaks{^uzU)3nd3NeU zk6m9d40co6OLG(3r4OC^PyjgR3eUPdmHf__pQieYQ*1cU4{Ajre~^;at*n}GeUnys zUCSBTZs>$D%rZpPuGAa=1{$R7!G?mB7ukDPlxaf=tNW$pb>Z$jaL}Cf|vL4RVocRE}w@ zz6{hw_3I8no&aY2*dp_fLpk!cta1rS?Jt;)CF0oiC}0uS_qY>vof~L(#ZhCQ6%9#hanf&|~uT}{tl3XXN)?Ri3lfp6R zvWi*kz1Q%>kf&T(4)jXP!qJk$D1fU!$BZEcfi+Az7 zt;6XGkh{T;2kBmz&1t)7XUJW~vncMLjIamyz3xenaKsl5WGUWsb! z6$ks;X)299fplgj-7S(LCe8CA4PUgCiR*v3^DZUa_QQA;Miw-ZOspXqnxEH~Pnpg1 zOw}{xDK5XKkX@U27VK38+ixs=?A4&mW83%eK!((8K}lax&dCMFilVW_CZci%6cZ?LP^#By(wlZQDC})PR@(AfxDyDuCp7 zy%e^89Vrg#7kV5%0_<&)wz**M)!AB03Q_{ji7(E*&4lm+Ubojcs5ciddnHF9xXh%g zHX}RR&zzXjeFXMf+^vV<^6aBhp!O1+A6xUtB{R|WIF9oQRlW7OeY!z2Uy5Bu_i7fL z=c?3*wB&#lxp+#Lr^x{(8+(o7i8EV&0y+&JasxL zl*3EvXSU)m#jMWx-g11fa;Kwf>LmaonQuKwW=oJ?7v(4l@)I2DM$3L<1(!<}tt&&S zBIb|%177^j`rl#FjJEqHHm4~-ho9BMNX>g=o7!D#%2g#+Jk;3g(nqh#q*82cB6!3) zg3B=pQ2WbFb-I_W*l-EJ5xzPKku(mK8USW|DgdBk4qx&$z)&ciH?|?VD@X#bbnfw> z5Ug2AEUCaz>p=dA1Fwg4q$Mem(<#EmL!#omg~T|iu+_MmJ(1D`IWM(<9` zHg>B&GwCgoUis;gXX9f`{8+~Dp)iCa7C7zrvq)r(B3qvdGZh7FVJzk*hc)Q4ap}Ex zJ?_!CN(|TTh@ef+$E3|PaN#AllFYu{>CIOVdQCe>ve&9~$4`mth@3YSM)g|b1m`2( z{R{EuO7w=Mp&vmBn6>C;2MFDlKT@#WDOXk(d_|o5g>I&sX?9`^MBC83?cqsRTiEGJ zA>P^?V?g(Azc0K1K`;&QPcP$7$jGw)S8Mz?x5!*i$CC!GOm-j^o#+B?vAdjmnS|_D zLEHqmOZ?oVBU7csd$AcbhpxAa z*Q%*^6g4ZaNXHE#d^>dqR24;-;XH(Xmh8FY_sK2uZkMP&j|fN=Y@%KmILkTyHJbkJ z(~xrJ9(I4?v@U8{f6%t+jq9f=^mQOc`l2!J%cE=1vQg)`6ttt-|MvAWFs zeo?qkW%s0fp!u~Oz<89%Xt45>N{dojw<46xU7cUb`WXXrX`Rkbjeyw$lbq)J@KhQ5 zEf?pq?o5su703i|uM&{bh(ChPbb8&yPsPO8J|lGA;Ld&Fhd`A0de0X!opeL@ntX? z_=Yxx4%jXmXNg(Adq`b*)BE-t=Bm;;_;} z^(r>8*N*bt-Ysb1&G7~#e!M0A8eRh0YWxf&cL#D+v&6UGu`u?>3%UJ?_M2B4xrlKf z&RKJB&*cOH&4$>!Ev-O?8y|DxM`PSSztBrr`_K%@jSCq9H4){XqmQ0G0L0ntj<=OW zTR2;rv?Vz2bN{hZ#Wh#*Y@Z0L>?^$V5KtLz14Q|h7&ldcH~*Tco#X3U{UuSQCw}K~ zwdy8TSrK%~D`Y@M;YL2EQP&cb#z#iZ?Xd(OLodTd&M+U{zRQ7R!mKq%O~kAUf2r`V zv5*r3ngo*b0z>vol%D6k_ygd*{iysiIehxH<#uPbMf&bWX9zfGmp8lhN)|0#u(V~4 zW7pF@q1ZKqRA$?V`TTZ*Nvu2Lx`5+?HMW(v0F1`mWFYBPeh z0jlwR;Q>{5iJIoPWJKu+*LJ|2Lq5AEDs<%HDqV3Uz`bOX*j3e2bycx0FZWOwW<9_2 zIWiz6zCNYv;G1^t?R0*zGSz6n8NZPNQcj?9LSqBm&UdK+0*FWKmwP&5I^fw`6J$q& z05j&8{aZz1qM^%1!^WDy?iv{z^0jr0A>o0)|6*<4kSEj3iqG840ap>-*OdRqX_TKP zyhNveu0%}Ni7;DZaMgCZn8OKN!T<$_5v@gQ&$WeH%3Wq+Wtz)ceum;^stce#wqDbR z#Ea9`kAUR*K;)15qYFkMY0Cgn$D-wpJRF*h1?!%XGv+Y^3+KCz)c427mL`_I&JGt zh`Hr8RdC!JV*o-jsl{E-OVpnM4ak?K?|(=Xowf%a$z4kaYEf@^U%bA-2iS*x(PQ@4 zny5Wl4gaT_TiKD9!EKo@&h~%F6hpP@PRN;vK=cO+yv&|nauociiG=1$spg3n76`6; zQn9^#k^b1PC8ISj=L26kke>Y|j2=k)3{c-MZ&njmDgi@~`5-z`?P9B#4_ovFFij#< z>Bt4w%s1Xwk~81$Lw+lmX8cJ`D%?p+=sw^Cn4|BwLaFCV(16NmNqa2(I6Z=f>7WCZjkJYwR#TJ`BwCWCb z>C8tcp3j#=O^8Y*TTC{x&xt42dHgcoVdANEu%t7lm9eN{H(yalng(j$_=4UD1{Nq9 z5bJl%sU6uAd(N!<)@}5*0nWK)*BM3^p4x09jrakeiW}>)@(!Jnbr-79f!;dvx@CZ| zW}s>`XS@3;1f8{WA*BB3Ff?qwdT6exk)6rk!?_BGkG(kK9%3VBJo9r;_$D2WJd~HI z#MT_J`j9+l2OK6fG=+&f*-IE`c}zh^9w}WF@S}<&?e>LVgZ`;#VWusAXTl!5*NF3- zloUUb`pV$OA*O0Cb~w5LJ4LoJJnE*?K09Hp8)NbbrQDypXxQMTuY^zT&ZT z*ya-E{s}jFyqoeN{6dS}bWKL3I)wvj#+$#X+;ef~z5R9{9UY7J;{c|yFKvBlY*~zQ z4&CVF4xwF3)vd*Tmn?t7gZ&neB#V}SW6aNmmEU|gMq+yJrKQm(;08NpR?_m-`~R3? z1$BOrRzFk_cu!t6ujFPS-KJPV+6ZzrjQq^gG+u&G+jqHY z8)TTi5+WQI6Rmqy7QM;2TTGv6-B8ETEZyj1A#*8H`0PH)dlZX;dh6*JOYBgLK(!T__ClqLqjfcrY=g?t^X>0%2u%xGhjUG4X??v zHfJA1ScYCivg4h`D)=FU8r}c=Oq(PM0e|fTPZ4k+XkzlB!&PS{ycqJ)tPjmB0JX;*ij# z)tS$qF1hQR4c+Obv=lMK@`m%%grpO*cUl>jc@=CeTAeeZlwDUC+;Hyz9^L*g52td2}ZVsBSqa@$UAN(e{Bkg6oXlID$N$9`%p?ruadr1sbpp{|deRMeTy+tf(}XTGyXV?MXxpYF-Peg8~BLg4M<_umBDcEKv`? zyQNOL8j{;0aa@@7YOTm>5r3rN8JuBO2Id|?eJ2!JyTcemk8L%kC~(1ywq^-pDr79N zUDhE3x@Y87OB(#;r_ZbGw#4x(OZ047m_L>=%y{-qWLA0zGn#beB1Mv29c-C#n8(pe zT*I*Ik~yO14u{6wcOIs(gk!rif>z~y4_A9m$N26TE7j}Y4ax8;@=UxPlD7>kAKo<6 zG|9~V;t3Ig-ac<(nUA%yeUVKuM3?lzh%vI`-wlo@MzrvQ+YIat)#qvBf$yy(Om~&? zeoK!tLEnSez93zK%#`?IWlZh4;r`7~6W0H(bsjU}KHv3+-9wah>b~o+^wm~2&Rc+( zu^Ro&>HuPPY?q)WEWKm1K2MD)+_l)TOyNlze%w>6bdI++ef%V9;5aeHy$@2K=^_zc zUR055c^pC7G~Y}TiIurq8zpvk3(kvE!@XFDT%F+yTNQ=$BmJam$04jqXm9&a`Cii{ z;{`f!g31&hoIn2A{#Ws?J%b$wbi#$uss>cN7Wdqz#nl_9eXWEfuG?8V9>R1RF{D@^ zeh|Hz`SEYlhcD_O=A_bc$1I{f+k81rfJV~pT1aT4ZAa&1^}kHm%oM|}qbZx6au-AN z^bv`O8c51TvBKjLZhrj@S!3O&?oJ7@`kVi;^H}NXwKcoRQx;60%(B)CC>lx`Q!uX*7mw?N1uK61#2BY zLm3b2-`N|k#6L|XWe)|Ou(~%Vb(~_98vY1#CyP&5w(#O@^encjn^((4-a1Jh=%{)< z-eLQI3R4o%+PTuD>E{<5Rpu$sOO=O&O6AS^ngVycZ$H9d=OlYWnNHnAFbFhYxf6+2 z@>*m9t7`b8$L%2U-272PJ->d4>@Dzu=yea(xd1!7lDZzXdA{)HA3yVJy$cx3UF8oV zp;l)yGf>8+5tbzRNZ`awNv2j^lTJ#z*TfA+JGb-9tDPb@gNGYbt-%*kR@82QfKPhk zI|wVYt8@H|kWXF{_DfK~K^4_Mk~xDTv7ApJCc*;^gRIr$xDdo;aIk(_!9sylM^aGb z6X0y#g0ye#;Zg^$a2>FGxlo0ebq}r?DZAsmH9F_go8lMyw9Mru8QzvwdPO_w6B`_E z=4e7j1n#>q3!>DlVxC2B_(Fz^d~O*wZ1g@EU`y!_OQKjCZsr#?lmJQL)%0lRR0-=I z9{w^*pyvlyJZ8|e9dR_Q{=3URa9j+E-MOQi&aQn00SAehVC6+6QkKZ%(CBGR{2z&k zFNmru7Y9Tz7{|xhjDa!!l{0&<|77ajBN_02fBl=KPa<)8N)qa4$@|PW2fi_k=q6U5 z1q9B!1(*6yw7+VDL;OuZsjK8qDqV8Nk*GB@n4vjV)?X2{V|DED*vnPp?EWj{7!`jO zaD$>`(hL0~`EKUTC&X$!TcT$sl*l=j9##Th{I$(bqTt;KlJ^0pR?K@pvF5fJi}TQ1 zshs)+Hw%{NPaL$58fMio;zPhpM(zaV|fetT* z7Uf2FG+*A!HX-?aFL|SUTa{KVAT3bVW`m>}c5rIOunrtVMP{WgtOduGr7FM~da2p4 z6OpAhG3mWfcRZfZhOYC*~?TFXfvv9EYcr4q1%!q1X9B=pgPPG?tJ#hV<1+l*)ED5gn3fSART6t~h!=@~CN8Pub^TWJR zGA24ue<{LGni)|q*uN)_naCl%Bw8eGCwX{E%yuqKt(~9tI)=haA8D^j`X&-8ucy7^ z+|;X1jFPnyp7RufZa++pw#YvouSHDM$~|Iess#1pyC@#5g9DGCt*fsNH8$uZLN^oD zYxWQNMdX|z`pDTgTBHCw-7WX=`z1-x43;xs2ljaL^zb!BIKtR-^Lv!{-M8=cL^JbL zi9DLA%GddvyPVIhK8gaWk^GL;=_uUKGgXFljfI!2P7>dt5Et%VW)*f3p6zK zD)wt?1fiwwsS@;)-xS^h)p|H7zbf4=*CMGX(C_G-ajc|=Y#~vp>qXQNG1BPG1T1)m z4GWd;psH;ttgmLY?4~PhW0hYJl1K)}m*SKWv6sw4;L53Ldp0QeRB0A4O)yhyu^`DY%lH(YLGWTQQkM54PhMR-&6t(>g; z!2$=_uyeMiA*V~@J8{H8B}dKTRV_c|e$tr&hofAm*qVjkY>6t@fth_SkISCERSM*b z92-BGqp&Rh@!w5#vf%pK$JYnDpG4uWYo!Vtr34t=Xu{66tqt;UW_`OaAa-sx2meVA z_Xp>wZBv_kan@$L#`SqzcEF8~%u{bI48Gb2- zHD`ejf1c!AJbZiyE!}p^;Zk0cnr*<1F>aK&j|5LXxamAp#-ta3hgL2bO4;keD{g%I zCUAKgeXZ#r);)(-y813an)fwN7-{o5u>54WBQWutS-;z*%1l<=$jv;s7>UV`!%btPn7A?%YH(du~-OGX>?qSUmo(h=8=qdZnee!ym zs*7YdmD9;xLIQVYO%fd~QU&gxEReQ!n2T6l&8ZdJ%xU@^-9&blx>%wjcsD&x`j@qr zeE3_Oy~l-R+LB&47K>n_BDPIplF)IhM7SsZ&eEVpGbT!k=z);Pbuln(?2@;K@OjxY z6ZWK<5Lm?KppSt6RtEIJmRhgrsOxtqVS+r?>+y$M1n(>Pc>v>ou zhuumZ4vWud_sr6?Dlfi?=;_9KNK-LuQHR>4dE*|D<8Phm%0G8nR@FibtqVYxp6xax zN}6>?oZBGhbEJ}vlL}@lRBd@*

dGY`5S|9|Na$eeer;Dpp?(kTWuC=g(Yh;yZus z^tArmWiRW!XBV5=4zV7D9Zs*6<*ZjcIJf93!?aTE@!)}EAhW%%w3iDUn>>5Kbzqd4 z8Gnx|{#{5xXUwg9pGskc#PyVjPeahXyCBo_9HA7hFN&)v1;sNqd|Hs}Xk2)v#Iv_} zo8a(Ae29ysGF%w~sP-@H)^|x~kk{b>Cs1Ctjfj%3d~~cKUlz{&!d! z08>|_{j-w*l<;jqolA1hu_Kjt!&7?|(jJW6#}>doU&bD{Iy={twi4XqhO-rmc~<|W zckE(e&vx|2YhYGUnOoIjAv3YC^fDWs+GAyPy@n_3uJ=n%qp94MA%V8k4O&nzb)z=8^PUQ^DQ@)Z6~FPgz%02hS(}megtaoI z5mn1{I-|aI)mUD2v67aBo(8tRb9uX_vVF(5@9!atnn?RCRJtm06J7R+B^JUf`8~Q+ zX=1ZSp>!ms(~!vLSf3rp3~Jxrun;pfJisz)%X}Jc^{vvWs0j1M(gm@U;vt{)YZNJE*8v^7)tSU>}Qj8UVU3M>AfG@yN=Ag}|mE zQug?EE_KGqU{tCi(-*L?z<=5En0|!eEW4Qg`fs-kc4Wt=S@i$@j$+8gD_1o)@avQN z;okZxE0RAS0T@R-{J|hW%KnoSbki-k*`I75m@KD)?-VWE_$mj?k*+W*i6DcvV=5kVx@2WR z;533dU#*Wt?QBHL=R3aE^t6XcjcOAXR}f6w35>4)3IKDaql*2bp?_I-7M4b)FY@W& zk%8M4?-E)L105935K@>u9~&Cg`uWC22GqlIquGa#ErSclT-y6i8tx(Fz?3Qz_z8=d zjaL0D3u0#}!vct>4>3h1~z+$yPd74Xm+Cyjw=e zu|*piA&8kHZZzFV)BlFxJy9U-B@hs)*gHo3G#6R4XKPucM3|xvG?lB9o^S~A+AqFc zQ>1<{g!yv)$k*71XA|AFt@bVkmURB00CqKzqV}M@%SL`~5gGt?Sc~{!_?`?NfO^T4xwn8gm zg0JFjP|MZO+58p>*w#4FxyTGV~KLeaN@8 zopyrs0(#H`M&4L$x?b+&E4idC7*awnsch+3hlABf)zcP}v=`Zaulc|J?utrP{+o#r zci{tAzSkBG)CO!>9zyH@*W5T0ou(uM^z1P3JDPS5(l!gmrB)x7->299;kdB9g|y|= zo2oLt4zZKghoninFO-eWolPr7Y+Fv;nsX_?uF!h)XpP66Cd^F|=r@Es6%Jl&_do?I zKKh@@aMa?fn_4TQcD;VvIY9nw`~6;FTiDBFyVs!lW!}niLbdZ$?_0G>X$xQ@(gAna z&?)oNCcu)-gw6DUqS$HOkfG#XRX2&gDZN2E&;kfL-Up@WF@F1<#jsWclPU8EB_1f)g-f>Nb~Ktd6b zmJmVx)E?UY(7!QZwcsZ4A%B1Pwb!) zhJu}yasthmMZf*cSP?DmEgb*wczggT)97uZR?&td5<-p_E6QR}Pj9p2^qNV?+26KV z*3T){XU#QRVrn)ov%G&uL>ae0j;lF5-jX6)iNc$cy{>uIrx&|Kf*aeKX&I*lkX|jY zEBIpEXX`vutFFlru8B^b};UbOB4TN$uW>ntHo2M@8MN+P23$ui?Z-__)xH*U(+)D{Zzz%nPtM6 zjUfp{xEr~L*+9_g5K3NHmcl_OE5bImMq^>w`HtIs?U7LBLB`Gb~GrzniZ;FK={j~ z0Ts9r2(K;8u@z0Ex&^6MXZ^$-1$lI%1cCFv}#ctK|`r-tVZtFeW z;ELvo{iewDECOg1srP*D5~aP-!YMavBc;cxfsAaN%qrz3Wicw{q+6N1r9+eQF68=5 z+D!SHJ*t6a=f|QAvv=yQn;d6ws5S7wz69U)H}GMw#&GVW!S8jlEbh9UAV?Sthn7x5WL-ZdJhqAX`n4SsnT5?VYIv2(0tv&;agsFAv@)SZgA%r-gfZ zj&)=+b_*9xYq3xtv4-gBp>-!!;SA4YKoxlbZ&+{g#|D521`Wdz)n^VHO#9rSUm1&YAR zCMfHh=DVFhs=50uD!X@X-=OGx(6MC%5%=sTmuRs5(x&&ksdB$qRJVvslqm~TV#KrW z;d8!O$y_0WDy)}zHm(C`2?fc%*p5Hse)a%RVZ0WQGS} zd!a&l+JbR!b4Addn*k{Qj|B?$Ew8@YemHxDLVEqS7==pds6Dbir_cz`RwdJHcTT7G zPmd9M@#{;L`sUkQ-g}M(x=>XTNZoM;Rbq8_tW}uL6n^~ZPc~(8D6#1FnRdHZ_nx*r zh;mCTxPAJ$h*-9;BV+N3`x!oyrqZ}s9xj7$$I}JbQhZimzq^yr?WqZk$_i7W$@w^- zY{USsm;BAp`2|q1rC*PJ7Hj<}{Bt46W2<0qgZciGtBH5cm_mSbI!*EiVyxs6g2qzm z-8N<8&wQ)Z784OxlOaHJ%R}%>)nP$8CuX#sBxwr(jbVTJy9vi-w6``PGt-Y*gn>Y` zjJj(s#BZGBr7;}9G8KkHom2S9Hrr3+IG%nZBGQ!7F)ng0^$AO>wV}QL)5}*odU{jh zW{r}HL_C|m3h0{O+<2wZ;+Uw?=Jjf3D@!uIoF%11yAp2716}s?5~6fjP$)}+CO>bRs>+@lrcBjQ59{c8f|LU9*(Ga)+wb?J2K%Vw2lY~-f6U0r%paZ zUD;YVx_Q;?3L7Q(D13Bv^N#tuV0MPh#&9EG=xP*pM{yXV<)Jk!B~3DMJ*?v!u*=cN zBMghR!%XK8Toy(uFHNN$9@AsVbV$~?{k&6;2bHnn#}}^C4Ar#M!b!Fval-dgld3Fc zBLC}n%_hBEw}+Hk3fH$KKs@DpY7zRE;{?wvGjpZSh9v~K&WwK%QQB?X|vn~SM9sCSb zagDOC)ygYbK}S)&EABR&9W&>MB?v+{XSMHA@b)kY=B_2l!C#(r12T zS(Ij6hX38k{9cv%BblPBsUT^VjOTKhqHXC0Xsnmtabq67;WBjI@=N`QXG_%MCt(I& zH(2A!v?>Pnifz^|>M zU_&Ved>9XtE3~hrbeT$UhmA|>b zag8RB-_SL}ASA&6*}dIUBu;y$ODozUuut1SD0o5}ExB({juO*>Km1U!m?u{^b8eA2 zNJZ(D?x`CYLis?qTw!~`d~MnBfN#u=)dlmi?r>7`Mb`q{Gp)93x_{Nm5acr>{ys6b zPa+Skhg%#l8UJBaQvTDbt7DTOuf-y|AyHv@j{(^`E)0-`DDC{Nog{6TJLVA%l|R98 z%QE;~*hRHT1o5IQhsm_B~ITvlrc8?>gpY_yBA$Y2hM^=1?!|&`}MS8H+vs;6eoong$KM3r2shg zUBvCzoW-sZl>A#O@4K~CX}|BQR+|FyMuI^WSv*M$e2=)h5rfi=A0yR{ka$Ii!0O?A{;X11|LEEO-J`lK5_ra0fyODMCOoa+uGKk>I> zEQujBOz5O(0s{F3dw)h7e_;o8I@ssHHSO#T(-~NnlE4H__>siSHlc>EbR`9`7Zbck z)aveawQ0oND^(ff;tZ%i9?()^>V4Vy#8hENml_*y=a8^|#oIH9B}1EJ24&absA&xGiC&n&0a3o-OV#N-Cm7M-ZTEX+;Lj^MpUx)cle|M4T>rRE_3V z3CB2E8DfO9N;UuLZ~wK(&VE*lwn{8-o!?Y*E)U)of)jPn%7LhwgaK>o5YNjV;H0PY zI5#rI7WqXCv+JW3LdA-7fcB9WQqcNIUX-xy#l-xs)6t^Vf#&umNkx{25{MtM7Wq+; zGEz=P{}X}HUR_lp1S$n>jkCq1-SfR=Ilvoa&cnw83H^K`?p_YoaH_BkEXS_uM@=wR z8B$?Sn9jcM!9q%o1&X_IuPI9O{E9JK0XhT>1ua)m1v~{+L?Bk9kr^FB(y=dt4V~mW zx!uXq7btM_do1`Dl4JI)T#C2HDXvkkRrVEP%PpwR z{s-VURHL38)5vu)4!7jh(+{^1*Mb>~mAn#ozQl{kDUEuCOJIu2sJtCRcR(Tr*;RfF zX2KWZF?fPPp4K>kj_tDATtfvuqPQ&#D;8EGT61i?V~7WT#pJJsF=|?~HX7h)I$TU? zIQ$))T$q7|w?cDH$V{f!JI_V-HO|69yrDLuITbyOZW`m}-|<9#_A zOeok&nX)F-Ounfv6t-V>%<%V2Y5%mS*s2TPzICgl_I_>SyKVX|QPc22ivj#cRux-u zg%Dxek=6YQz&1G4l{qoTh03*3D&k{Wxh3!P@{@N?gT;?_DUVB&>>UJ!(Rd9j@pRCk zs*`Pi6uTAUa6mxnD?W|nFHem=CuT)f9e&n^%)ecDpV42SPXa1x%N*BJdB1Ad{Db}T zvUXViM>I~P;zEZmN`Q%6oM@$ksZW~uj)#epU;WdeR@2-Tx3rFG3uUBdvcW@( z-(||hFXji?TerpA181FY@i)CvwQfzTdM<>1Q*QliD#(>+fZ@Kua@)Q#-m;j3rBoz{ zOS4iaH?q{!!eSSO1(ib@#w(i;tSAK%0i*LfXJ4-(WbMWhg{1Oq`vATC>Ex9^L;PrlfD-a~v9PTM5$nAGGOy&8vq-nO*)m82@!S1MG-c%3;8b28Mv z_LCv4hj{SW!h1l5t2wyR!QeHl?1i|-mmlk2&k*EU^=CAz z5;MG47*A~1p-9icV7o^|)#TbQl|{ zB=NPQje9aK|8|!_Qic26si~4Ebp3hMxwt9&<9s;Ud+WE#_P^sv)sG)LjFJaiLESEuj0R$PW3mUsx5U3XS?=7{?m!?Bw%ip*?A(4kmbmv)oly;Fu+^GbEyjzh`NT*M{C5W(BebsU2LL~Um z&pT1WTKQ3&)Idh!+i2$RfhHSu(b+h(#uwr19)_o@L}VwtPFFlUh7rC6yH#a+6pB9z zl(jO5xMhpf-yp`TK+#@-4Yt_?}bRas$uxp>K?m&a$Q8gl(0xolz-1LKy zKq^iwLRLUICE4r#Mv9cll!0Mkmlvgp)=BUZ0^L;;z1#gQ@fW}5z;#S2cYQHiStW%l zSWJHBEQoo9W>bWI0=025E(3xT+R+^Ac?C)Im{DK(z%#bq6hX?OD;^e915Wwzx(twb zOO~8`#6547G&P3S^Bvtzm2KaVUeVHwm$)Q*XG*Z(q&y3V2b-8me~Tp3ZE^j~w7}z< z@<*6jABqC*Wtaf-pS`Mmiy`!3673hS#0q0}Htxcg9jF}6rKcXx+TviAa_ZUcb-cfz zx`wyAcE+9Uc+*!P0{9-$lbf>kT&Y!<({${(CWhR;7Yc}8g~JM1yJuXYw{Be)+{^T-CCwoN2Rwy2QxU*PZ+#I!(`_Q7)6tz@2!)2CF}N94Ltg z^bw_EO_8>81=Ni~-(y}^KyfPErpZgUzMN$7c7DoP$hvL){=W9M6@Ix#LDva zoY^uE#|JUk1cc^0v82-%`+Jg?*L50A6MCn%X);073}SF5UG3NSTNbRv(((6SKWnQD z+9ivFaqE8wEZhB!67^w{Zr?Rt&?J@~(?(vu>3;JV8! z{ONmNYFDf#c4MMbzo)*G-r}hNj5axW4_ze&T$TrOcKmv5nETZL`K+NhaC}Ld=YEgX zgI)I=xRNUbwZd)GY|ImLE@<=$n_!OVvbYU>fB7(hcBEIwGu-5~XIlwM3f^%yE?-(7 z+*PghXWs75Y|X!?E^I#%BLrPa`12Rm@#GjNvE=-!CFu1rl%W6{ciuME&?c`p!mnUO zM9=zbxB?V)`*lTICfU-1H4as|21Ue~dwmxb@ynKssftj5R`zIh3^7Q)%#Q|6#yuNy z6)aR$q@piP&u3@-wANGMbxqq#P2@?I!VD3!(;p(y@djSUAO6ah0{ko#2u}ZVWY)Bs zT$-1q{yn3c@!W`&&rV51x#A&DlOA{8g+6_5d!W@lCogL7F^bwCj%CUzOSk6SN9zgL zjc~^yLD)IhSm1=U)EB*XyEs}BHJpLtM3f7)%bEST{B7vueNHycQ5=ZFLMM-Qn*TeJxsT^9Xj1#`5|IYd8JRV#jE*#PR~ z;aiv@0J-C!@t>9$>${oU4F$T+TAhY@hXX%Ap{_wqNaUuFWM9j*VWnBq{IP(9`uyM` zIOSlwS2iZ@e`DGzeVgNmtASepaiWLIt0M0H+d$nR6Dq~0=o04ex_Zd2 zGrSlBK3gR^<4`aKTtbNe>LK;v;EkAurvv{gH4DTYQ_?+j?*eC1UGsntv>8&a#@RS- zHzDMx1Tlr%AYREXkq$eWCV`#-&fZ7Nene@uKxWZI-Uw7}Y6Oa%-|>05J4q2rnz%1N%AI~hV!XW%|(mKUX`Q=MD>vM2Dj6#+O@ z97WtONbJ>5O5oJFDh&+z~*}tC& zADm*#91udAdmtmzd`@_;aW>Yv5Ed&ef^Vm3W>{=73A!AUTf3TO%r135>ndbbPk!TU z3C=(_sr;dO9raKOEn35PO{bPxyMBX___~1hLHY)XVO)OYBGIAvO6O475v2<;6;jDO zeMM})LXxGxeam~H=9lI(dO+C&*?KVJ9guOMZ_Itog{vWrUV@|p=a;-t$|;^ItL^{fyP{oueS)u?#jVr6IjAbN>Ie+7tN+uf9GBCJdhAi4>i ze8$R!4xrI9f&=zl)JOpa!|=kWA%5goC3}L>M zzCa!FJ&{-j?!~Uh_iZnK;58u&(3-|&-&ZmYOKwm2P*Bb&I19t-aKs7an<5$fZt-Y0 zPlDhiJ}aqjp-^r76RVq3o|s#fEX9=0HbPXd>S(!kvc}LG4LEOC3S;^fX%k9-OiUun zXo7dLWacwG?SrRqINb1xD@*~S6qL;4rlWG2hHlJf@D#XyiIAM-B^73*%Xat{WJ=vw z^Qm$_aI91e$hMy%^yG32O1CPI5&OPz$4*prvPScpJS({MN1#uUes}aHusz#QBNl^$ zUs1)jTDcG`s)VDuW}sC7|4+$)~5pFbUI5(f|3G+kd5F^;pC@iJd5K;0JC1)0Cj zk&eH@r!4ziD1mDpLclmIp9hn}AEvZ>6XS9!v5_K4h7nR2n>H6}+uquUUh7$h>)!r( zV|p)n-0tirBp-uO4iM%zP3O?wc#}0ifTQp`;97>r(xCbCFYnS%N}>GLaGjnF;_WBN zQWtlV59hiME4}r5vSWAp&<{SQ^m6*Ozua*87*;546`u0qZVdltkDR>e@^9(_%uJyh zY^}ZLti-0A!|cOjBujJ?+?w2sl$@s%Kl0vC?|L7MT zsI3`g2MohVj3u2bIUPTX2cH&-?6-|uPI3Q-iC?E9_r8rYy#2b`vMYYp`mCLDf_eM* zS5iq6)}=sDh*gm56;f>TisOFJ_VQNd`C}MKZ{qB`l8n-)QrAFwRTq*g0WCCgPR#uFCRa)wZfq%uk-Ki-Ib~F&}S4 zIiI+vSi7zqI$#gB`4K2nv2@i|iC8rSg5jS!%%_9vPEHj{9JloSm1Xl_=XdOiw?Dx!d~2sPG?(Kw893+uSUtSFf63=K5KfeB5d$xLdPET(3v#jm( z-7adF9^Q*CyZu|okP>!L2ma|)o<(w8QSZaWvzh0yeeU9(Yth4oH!(K1`QH}XYP*{z zSS`iR4n!yGiT_Ar14gTMK|=NE`fc8huGa+7gpV>zd``<-`MxYgZJa93Y1%~K9=?J$ zy=#bOc9J}6%lkdUe&rF>81>?K5Ks05S*m>7HFSjvBxQ^B%IdUyK6R&&ZW*@zqNv#7 z(Be+tXySDY2)UMi_Q+*WsY52DXaMbA^R%jR8?&mvc(euUAh z^_v@oL?gfHxu9^LRljS~!N3I;tY{0ztU0xjfvU7eZcJze$!qV;y-B+Bq;ldT*~0Yn zovrB+B7Oy3o9ws1@v6fAoSwvTXvOTOJ|{_(y_V6{RP@)#YFhXg%{Hh0BE&~6J;uSy zCePW)1wr0uaal-eq=9X{F`I@z!6*}*^>{RRIq$vc2bjTtI@Nd~=b0F!@TAWOas6(? z>SY?j;K+*kofuT#y!7cEMrEZ3PO=Rrs+p(J35pM#QuUFY9D1!^BG*=w;c87`PDk!c zzoZ3MOwY1+>kr<^{9GHPZ}<=@p;(n#;EFLe8Ia0?=?rv<1zqu9*}9}6zzoydXY^e; zD3E@P=hT;Os-_{)XVDo7B1=^!YCWIl2q|f{4prtQuvPv(mu?~JaVqwSD|kB6I9f^au>XNM z<*K&6S?6->4j5r%cX*s1T+BRYxlH`47_Ws@@b>8+3}=4tkUAr-STV9yPOXk_Z$9;f zH&inOoElPfrF{HN^l+Rs%JW`fGzTbg#`eK1OjWmm?A?c0bec&8p%hVq4Ff~=+*zx( z&b*#@fiFyG?$j1VRDYW$Ore$z**%b7XTGT>B-(s`1)a%XZ{^b$@NkHaQ3Fd=(QsWG z8Mz}W3_I~&MJZAIyt;e<7qr~p(!P#)-TufM<|QlIKhR+nHmt(c70ly>LjqZy1DqxJ z#uFMgN=AJFy3#f#f7nUiZtoz!Z3IG@7~v1ZpD1o=)SJHJwbEjAa=;tYIOdN>YiFDetuU-H zFOBF#IpSQVJ_P7=VZ$^=G@6FAvd*R2_no{vgJWvUZv7l{A9Fq4 z9SxB`pjaaN6pOD;rZx>Th%Xp3`%qvOK!&~%r>!GaphalWeP2+e-laiV59?qR$g))) zt9!0OM6{}Zh9#p*;P-srR9Ss*b zRjJ6qWP3T8jmCD{#O7LKs#kNRM^vJTA}L~+uuAih#hI`QyyEV3g_V6^Dm-0Pdv>Uty^a!q%V^gzMp+tj4 z&9VcrEne7)LPEvTJHc6uQ!3FlGbhA8=(&M0$L0oq6VAyY;4QIO-2i7Mg$R@6;ME+^-q!ug~sk!Ulp$oBM&;V6$O`w=>>6l3|jm)_QrqcnL)-KR`FH!y_Ey*Q`2@%4$r}>^d+6g3=BQ<8UxNiaafPD*CmhAbeI<~Z~8wkeAcIa4>|czMent=E>+90j4^{vZQQ+}&yrIq zM6{R~Td%)Yixq2^H;uI>L3#G|)>lW>;25E$>SFmb_gU-2Ia!Jdn2Am8^Z~jh)&aWm zTZD{kiI_Q|V)eGk3_rb?`aG1JaXZ(CriE2|atLIR^t{K^{nkMNOrz&kt4aF8moh80 z#r~N)-wu>SVSK0=vZ^?}hU4AqEX#2L*{CXz$xmZTUwRV0JqvS8^oL}I+zaG#dY7&- z_9l{CLQ)N|DxUiH=3^LR9&ieYag~S+ttfE(qsDT)kSY)V$d{)w*3)ctJ#~hu6m_#C zD2tgDoDo(*gPLQ3ZSSPO&mSshB|8Kzwg_Zm)toP#`iix6wzal3FB6QZA6ahpfU(Nr zV5&yGt@Al+`zxp`xY7@4QWcLL8S*Y@bjHD;xqvzk`RYq1OS}?mV9vH${1TQ<9nrYW z9Fed?sEgRCFqF-Rho1QmrwZJiTg6}h#!4oKtRN41Z=Zhsbe6X0s|-}JHI^KG{KYCy zSmjuhYuiQFH%PM*vrV2;A#O~3D4NyM4S(m6P8naiHlv1TN85WGW8eC3;&XD?PmD~Z zz-sLK7C;{y)XvCF-0& z&FkXx*CeFvVo0wZ4puZ=`B1omhNUZBIbvIOUw-=F@8%LZIkW2xrgSqc>pEwX4wcz> z3mWz~8;7`zJ26Zr5LbRc)YMU1#$vQ-`OqwFRBb08=4Od(8{(R`L<93{keZ~rRxY<)GwxX=tIs+LCGH%I%LlVm0s2)6EG1g_jJH9wtOLGM3X?p9Z&e4z`UtgYH%iDhvow| zZ|&`=s*^>KkC5v#h+O4<(H(Vd}Z;)A@lxDgtHM9s@;&g{UYK3C~P`cPbZWwJc?O- zy?S`?TKm?{r@hI)2JJw;!yM(q?w;s6f7khM%4$)l@&nB*N8PGMm>+rRoh#B!=&AVY zlv$)AHT(as1^fC^N>Cqju!zWhZ+?$YtX>;$h5xig!^TTz@ZAEKE~`uQU3ZE8&L*&* zG_*9)AOFfwFd8CT>uS&w}MpXG!rMI_e z%qy15fA#fut`gCVWnC1=p)V8v$)xsN!7o)THIszC2W8$9GbqeBD_&&9<&ZsdW$1qvKj2JKb>1h@+GOuHWCLN2Qq_C0 zi1F=BLF0g8q1ONId6so<;D?26#m41O$I^m)8qxdW?ACmhWW%XEeD&Vq=KOIEJiYjG ztHv7hIiCzxOI`i(y85ng!|30M^;OyK-qC80DE&nKyJG%$u7)osC?1T0KAoaxo3R&CJ_26r2Ns?^ zFMgAPeCG#R{^spII_!Ffa})&JB|!GqA0ux{{;A#9Z#ccsIfgL0Q5d}!`Ta!8NaPH~ zDJoH5W7m=NJvaefrfuD{*g|%_jJ=rh5&LWCWpG9JdYm0ao&LABKM_g%5wZZ14VVNf z%{)<75CGHXbN(3XSx!Fh^N!-iDPLUHEb7mgsa z-5!ZZhF9Hbt_}`T@I}3n(xm_&J45T#_Y(dit~=d+<0E6Q8Ma9Xqwy5=oQP#1xO7KS zu=Vq7%H42uvjWL&Rn3d zaXxUNoj|nBL`(E|yormwfTro5ApW!~{XsLTqM;M6qhA=ui=`r(t`UsalnBG64VO~u zE~k*vJ(?omvRAiSvr)}oy@%5M%WtX9Ki z=Z0%4U{gnwl$6G!T?4HQ>%5&EJL%*3jZT^`C!s=j4u7m@qKS}uJd5{n|eXVH~k1=tiyBe@MXSf<)U4OCA7*+sV6>d)uinYF!>UsO! zcsQG}<0Xz!FVWvItw;fEIj5T2`Pg!_OPNTwTt8;113C?zJuiBGwQ49qbEcU%9492ukZ0xFz$nl zMe3|4o}}>4H;Z~bq$*uPLCC14cmb2UiD}9f+e-0yGiD*b#;WL=e%Ah{(N@k_t(q% zE<>;WId{w=WvTC-DwF|E*Ha;N`z=p}F=)%UXU);N`YO#Sg>56t*3hi<&Di{bta%T9>v_N$IwA{|jDtfgJn z>c8H=>@VDHU@L7_7yPV>`MkI)-RKrJ%7@*0*Lkd>Tr^Upz4WhdLg6In_-Ftj+8SjZ z;2bzrS^oLur3j2GTsD4|1=L*t_C_4N)GYBqx(u?j?M6fz#QGn=C)%bR^_cFWO)&E| zTA%lUv*l=~bL48Dh?U>=9iE;{u;6I7=}KYo#-kbAn*kZptKzj1MJ{eFM94?sID*&5 z-wDwqf>bxZ_-Eijqw{)9h`-P0^1!{=ou#*u5p$YXg?u1mPjY-{lM43}yG=!{-sJLs z%8cMRfmZ;FbffJ4?G$B9J$sbL2d!e+XSpD=3Bf#t7I3-vSz72zZ8^WJIao=M^LkHg z@Tqoii8=e&LV5W_th}757ZOy~q4c2;8?+c{;Z$_gk}%pZ?o|?CCeye|?oDVEjp;S3 zu+bbWvr-Kd90&&GoY*LG4RSmNP7fx{E*yp|94;e(6IVBZqZzdqqot5LE4lLQYbDw@ z@-*O?uf?5>P0j0l-Az~jfvnHOdUnVb9!<&y=3+V))E65Zdqj!ZPMoRYR*D^en8xGp zq?N3C)_FH@I}6=#is7i=K(Y{|+xCFNuV$rJy~(|mU80wc_aNvxuS?^`)IxzlgG1I_ zE`*djAzkJ&QoBF{U!dW#rZx~0K~9hf5J+L>w0ZPvMrvEe?cl8m&e(P~ai_;Y3$1di zWecCUVwBSBnkq zK>az_-yi zq+(}wwjj%QF(YCT^yOA%3?0jpFQ?JO7XVI{NmYeNz|j+CszyLq{_rswb|mTcSv(hH zemYpN>$8AM194zo#=)9eAHLQ=_*f2+50n280n69I~?p;Y@>UZ%E=hw8{P2VL+u?z=ag@*0emQIZ*58Hl6Cz;12E^9|^B2f6)0%N? z6TfBn7Ks8k>=#nZM+gVD0pINia zbjFPT?m#8P1l|*`7C#g*gQ2zJG$k6}rwMj6ttL{Fy$2p%dXT?kR^@6(o- z4KFIl++IeO1bYqdTud-MSzT?5EHbHXvF)qqlI9CsP9BT0W;XY=9? zh+^(~iuW34G-$P_7WGG6<6iBm7w$&UKQ*sgd_nOBtuhN!D7$*xlN7hy|lE}5|rt0{n2a=kxmD!1`j zyTw))HzB{0LHBX#YL_IJAkGNXj*jT6WB8rRtlf&fi#%cyR+V&ZWscBs#MV@pFFb%X zacF*Gh4l&ae$e-A?XGloIT0_LV~53Hh^nxVYd-vU=_B9ESVXY2!`SHT1J2#`=%8^8 zX8*8f4)mLj5*)r?A1=iB)LUhhyu?O^DuTm`>=V=I={RIggCOzoJk3b8L4HXelF(Et}0Sh)sSNu|*qcjul!>;+4SmN2<8^61!Za>bL|TrR%-N}v}=p!FM!+i zyj`~sJ4Fmg>NDZ;uhuuc2wLfpEI_tpY39G3zSY1I7f&8aCmS7;zeN4}I&<&j?jt*=x4PAx#lZMGFQzZs z_?-;4dM`{F1pI0K{&ugnEA9J?)iZmNZ>5vxY)#ZNjR2Oig=l_qri1#&(}w<7E$@rH zkK*!=cZYum_J&$WQnL-VH8^@%FSsv-Z0>oXTRV@GzXy+ZmYkniquMvJRwLIDMYjw6Gar&IQWuLe`GJh<_2zBrkHF`@1hOpj$t8!E1HE$kV7SA( z`nK}5v!v_in*AQ`112i#FV&5@k@Y|P-#^2;GAVyR&bB}0{9FEkGRJ=vgZ-A@5IO(t ztO?U-8GYF{CI0_`)_;3UvjUL3x8gc~{Kakz_pkq+{_y8rp7>iDhK1h!nEZJn!V5a z_C26Jn?m*LTi!l|(XTK6kA5}u$5dXXUGOKmHJVcoH>sY!C+}DO=S*e}`W*iC=Ic}^ zy7vG2roV5T|JRrQ|G%2s=h3e#D*rv=-;Q-x>W}IU{2J7o{YPy7`1OtV&61lpwm;u; z>HBB*$7{aG;7a*lzkE=u=O-D^d-GC_0?rV*{p;uZuv(Mjh0LWf2ETp;{8y;kssQKC zz-!I5KVPQf^V`cFxH0WJdx?bjaDMroc9Q64GlJi@Czrx`^l~a=4(>fGgd)M~THM-c zG6f~GFlxKK!QOE@n-bQtJh)2bJOcMn81(LrDz%wSDYY3wxOG3xSeU7*u^c-N57xCT zRohGqKKk=wJYs}%DjuD7(uXe=MZaGeK$rFUx?D=&FjylSeEn13UZze-22V#csWPJFBKYl>q&a% zfyzF1rDB?LKz&xeIA?=qg}ZNug(lA79qX@)ixuIh%Jc0j(A7IfO6Cm*Z{iPZRL8GgH9wo2 zG(g-x)L$vMH``;!Vbrz_54cKZgYT3sDC+t7exHLxbi5xbbwnvyS}Akc_HBw1WeI-T zP?f7^Q!f5Arw5Blk~Gn+G@986)aJmkFfnLC#I%LvDJ676z*?9IL>O8wW$OhA#yeU# z3MllcZL*Yb0y6S{=;UC`y!Y-yhk^1789*#_GL@HyE55ar&UL9H(33O>tTif}md^&5 zjvx&#-kjz5oauKLgXXt0+fK|~=D>*{Wn0^u1lOXY?l&k|1F5BoD)L~focR3`s1v|5 z%1rG>zI@T49N;w3%{tCoNl4$>NRP#>lwePjhT62`Hom`uOpI;g$G%BpT+o&F?hO+6 z!L6ke4s6c2)Z}0~=*u)!hA)1}=JK!pA+OMiFTV9(^fPU$U0&0Jx4)O4?wudIB0qOP~(67BYk~${}_Vw-8q7f4~3wjI{?#I%^IkM4M zhbi`Xrf$Qm@mzJ$JwH)WbPatHYa zqTY`C7dc1lL~q`&TI9dwpPs`?Eyj4qSlH1rS1rV(Nl}+Zl2}1i6t37iCmkJw`~-H+ zb>dp45(m5czL3b=TTiQ1UcQ2rEHph3>b zaP4KO%|`V209$T&ghcnzP3eP0XqU-YAzF2p9T&rjZ#nPq1=*-D0i12Kh--48au0dP zxyvXlt{@>snvvS3!5M@?AXjXaa54fKNI116(N((hfud1ZaZ=ZH1((_}CrM~}0}pj& zBc{fyj>Ah45Eez({w6H_w1XN+qW1d2SIlAZr};&LK2S{iT+oC;WB|d4fx}QDbKN@unOa<_u!0NtS#J=FMtjm>~xK)ly$NI0;7w53fm_p zn-e@BF8*{trTeKy1t z9j7;BZ?GDtj@X)97S&nLiuJ+-pW^n-P`OsA`J6;Xbd6QUW=_+pJiYse4ye#Vj%#jI zmgIw}c7+&Lc-joY`{+Q~Z3FzNjH?9sFKDcAoezOybXup?x%2HkOiLvXjpt4_K8r}5 zJ!dO;rd-y2+xxk)YzxPkKuOisK0hqFJwv#UZGYHIdMrF^PmwMSoV zPI$!+b)QPq)Wofxbi71JyL>eTDRs);mc~|ZMdjhZknA4U5vKo?dhir2?b3jOEEGp| zCT$I}$slBGySW&HD^k~ekfrk(aS2-TtN^1{)}8tr>EHtVKOi z&UwwXn&ACVtD3}v$5vU%5CI1wQD`b76m65B*19^AEjz8)!a6epZQrz25HL=rcmHVoJdt}u)xWCop zbyPOC;$rU^OVHl!4qh43Z5Js@eqlDRQWabWx6TNd4QF$)699_5g3{)%k_#WmrmC*L z($ivLT;O|$I$}>uM<+CtW#joxVH3G%*}`BR*@2qrG&U3sMlRCX)jDC<;lRO1K#ta|!e3ZlN_FZg5TlbW)jVizY zDR!1TYnt;>qIdMn@Tz}2-UbEH$arcQ$J_h9aR#aGpvBR)t1{)S$a3sa4BWY7Y(&ux zSuv3mj(FvHA$v79$Y@_?km2=ANIUCz&NE`!;pwnlAJ2HTDabLWm4{|2Fe3pw9E7U` zXaY1Ml?6bBQ}JVdHwSC%y<3GCuC`xj@Oi^JH=3}k3!UL*Mt!vn&D-LA!JpMO4`%pd z848O@7mF(3R%A^a!6FZ?TSL5t&Fp7BLUtr4WstMhijkkSdMv#PIon>$tCFMRiwNaG zgHi>t73s>}-;27;+DBn~c5s;nF=Sb($$G}ulV*QtTOTuNG`R0U=OYHZlEOp%ednED zK_7*#ykj!X8?IXH;*u@y0&P}4Gic{URUv{UikQ94X9;gb!g#z*EvN4hy*R4qXdvzH zAN2yvrX~N#iYx1zs^#}?4FLwGVspsxMYBkis*wh3bHn?a7cEOFGzUb^r4(_DOy{8J zvU9u|DAWX_UJticTDvWt0xRuTL(LMT2;_tY16aDJQIIa$_GnlQRw)#lgHjBNsp^q3 z9}n{X(4=56E>I2y1|Q#pwX{1+lV=#KCqGsss8X zx8WvNKXU;@r^X)hWUf3Wc|fcvR(HfHF(!HSl5s*zmsq7aJfcfm-jQunify0mrst3G zKPbC{IL=DHQz?a!X8cN15(w!d?}eB7zZ6xU0>7Yn_E zP$@Tl^Y*>~wAg-cE;e;%n>JE)2R( z6B0%)Qrn%wu3xK8@FZR9XA`uy_=fDOb+ULHtb@tl(b=&N%8SlowokuUzNuRiGj<$_ z6jyZRoi!XH@e&3u;UuYFfV$-}-rOfcMO*{+W+5``d{X$3mgIQo`rR4>2Z(Zyx4qlf z4C#g=wqg$L1Np_+tUQejg?`-4^%cJKr>FWfWFAzQF@^y+g^M zJskq|Y+iVC*8kzKW8_=rb!uwq2Ayt(gEP~j!=DbFVtR&_7V6hGl5a9ft*Fvov$Xb} zAGP}?)5n@MSnJ72K2NKT$=|r=eMUj9(T~!B)oj^XhykWHjIhlxlc9Np=iA9#~%#Xd&WgqwJ{vbnpFG{g0-`_-A+x# zpS7T<8Ap^(LDrzlyf~;Y@kdoHx24r#3(|T7HF~;736{8WOKtg2;RQx(1j7^Jg{P_4 z3N4s`a|y;vopA@oV@#$(M@&w2kua|hdkY@%8NKgk2zOB)Y}kBW>V+c%zGlO+d!Pw} zxRVipUZF%1#VA_<8a5x24v;%BSNdIzENtz9a^1xLPif~L%~qPm@orHB$#$1gYiMUo zwK_A=)dWe@95ch2)!-O1A`L=~2#MeEBTZHj?39`rC8!^5x1u6#6XV6g|Zu`jZ< zvhIfITIH)pHr>#jS?E>8!Y5Xn-T*5`2EbhZ0T9O?q-;ar&ptL&7B4tzqD|Z=4!wF* z-`bZ=_uoEh|J9=~1D(d`aWALkb`=8Hzf<6SJe9bkih*&gBaL(X@2Py!ZqL<5m~MQva~p zeqbA~)C%O0Y|HR1m)tCtO2d{ole@wJc#md$LjpBjVS|XQeC7Ar(q{+iw1B0U+7;$RiLNbmB$uesk`f^+o+ER+{-l_b9AnmtUNECvNH2O*6y(9V?P z$tGZ}+PYiA?6U^I!+uB%(3ALr1$*I+%;VdULCpuoG~D>Ox|&pZQQik~5WF^=qS|-Fc}SLwBp*Lv22v<}o(B4Ws$N ziL&(+!rXUE`NjgSKk5o_E`EX=_-7l>#2GRRl=0cbaPpo8UTpCB4`7gTQN`s^hVpS9 z?#VvK7e}$=9w{FpFsdz@l5psmrr<1IT>mXzbT2BY&DZRMQO4u9tier;JnDXC*KD2R z!F8Ja!Gc?U+SJZWGq56S&8l9~Ma9ea8+v~5LP;lP?aX7_C?)gd^q=6^UEFq5Q^~xc zLD^3V(mpRZkq0{QxOQDPgE^ly)e-xZ{{ux4FIu+aC{tuWC`PAEhru8U-`kJ$(FBke z!2_m2RtpLA)Z{jN8E1^_Wa*v`p2OEop5$3 zv8OL1>G>fTe*^|Z%l=e>%S-V8iiF2(%a#f?4<=TxT7Mz)R_bs={kAkr^$xNyy8&o4 z2pm2buoV8}+NrzFnww*$^BMyOehVUhCwT3B(t#BDTevi$cu ZE`8=Bxs+@CQ@d06{4>u_@3{u2{|o(DkaGY4 diff --git a/arrow-data-source/docs/image/performance.png b/arrow-data-source/docs/image/performance.png deleted file mode 100644 index a4351cd9a5540b753774bc9761bf0fbb5547a5fa..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 81268 zcma&O1yq#bw>?aQbSX+BN=m78BZ71bC?zQ%5<^J~Qc4O42t()4-K}&;44o1KL&wnZ zfAId+|E~M3`>pSrwV1VL=AC!mIL|rz?7a`6>Z27u&kt2q|wmIA_*?vfi2PQIK0w_prKKb-TtBXvK&gH zq1|LF$VzLw8*N<$8&mex3;0O-4qSL1&m*?H2O)zIoF!Gd+VPHwhHJ@?L16@VdZ3{} z8khE+$S*H}^p_uF&&4V%pV}jc{x2FuaY?EZtidWMF=eN?zM-JtnndGIgWGNjFW=c4 zWF=pI+xO`^AztBgoo|doMbXb5ZOYl?xXDVmCE{96k6t3|@gEB;Lm||kHk}^rV6DUy z(&Q_=IGL}38c+O!rV-CKe#)dr45gOP7l}BpGiqpPcql&N;!;8MR=ppVL2yj8`1#8{%57tk5duEawmF?)w?-H|mdtqbbsPw$3- zk4HvECc=8KG0F-ppx?owB<)xA)9S%Hx6SBR=+uE+z_gz={E@t)CDj@`HxDge&JaJVxk3ST4^aoZ`R-^AZe`lH(2 zR1_^rYkf0m9-DGNd^z`XOMUb;Jq$jmOEr6^aS>jii{uj#sqbY@Drv(xnOI;!<_GPU z5NnMsK&y^JeirwrYBKnHu(}^_C4QKx9>`v`vNK_os#s~cx!UD#Rv*1z+U7!Ar>RC} z<}Y`ToNQN&{Gm%~sVWaU)u)T*!g-2}ANr|p^^pQTMII{JUN8@m{0K0g|+tDh<}3-)M)AR@d_a9xgF1;uHtcp^wo^{B@kjh95P z$uBpJZ$|2-`QS_t&gI#1n?RQrLl=7*1FW!o8{AH%A5SzQU%T6RXGbvqsQa*@^<%r* zo1Q{TWi*CW?daatA2zv>Y_K}bTP8+qNnbzlBt*aYPL?tGJouIS{Q@85ebE(j^%_+gIGAEwGJ zT#5yrSC?k-4?>w+{xs8hc#XfEf*$F08Y+Be34UmsERyrs~LFXKI}4S}6K?BYsTP)#qK@LeCafh2_BaqPI0LFFSE}D60xdy#0rT z>YKU5lh`+OP3el#A8IH4X!=G^un!K~Y`IhI+TrKc|H^^wUTIB{7#TM73aD>(etv!;=OExL=CqsN=aCm~^BZfiXpJQ3>`ylT7F&;Q z^4G*-*RNM(9Y?ggZN%@!hfuk~9v_#VSp{Qo34M(EZqflILtvfAWbFKqd9k{51(g&MKHRGj{`A@ z=QNki-ErGte3geGzyg7yHR(0_8dr=Kfjm$~K??u%xtokzRB^V0!Xxw~5o$7)_w(dO z8;>}G4%v#SN{Re8c|;H0e}0^5JcEnz4Ne=O^(%2%3xbgMMcHx*&%|% zm}ae*&6(Cuwej!YzvjakFKx_Ee#%UIGvQ)#oR|Tjj?x}jVqppUDrwS+#&0;mg72Ps zb#c-+S!QN&eR*cXxH~O1!)Kdobsw#<_#Tf@;rPUC9YjZ0X0ni*e;lXo__?4|{C;%w zYSDAI-FdE0pFVlOEc-jp&QZ1q)u_0*Y`N1C_Qv?BG6H;ju>bj)`T*@9I1|3d|FcT` zdxIaTx;T&d8?`y%Qal^Wu4VrB*8yMEc(Gvt1a79fOef?ZCH@*`f&eYowdqc-yY2Ml zFOPG_Hvg}Jm9zCON{C(}28r2KXFJ1pt&(bmO*Vw|VtNRUc7FPT{blcxXO(gbcPxio zXT$icM;=vCc%AM^LzkOAjXFw}TQf*N)|62pw#59X(|t39s)_hSlf#k>T#2vvE*dw> z-Me@5p=dPpa-Pa6Dsj1LsjyXW?Oa@mj42M+goBI#9gH>Cy{P*`an5d#&73{z>Xkek3XbUR`+I%+aH|*bPK*A^b#igs!dE>jsFMfQ; z(opN6V;e=wv*O|B0U}J zAlZYrXKD)CNQ&$}I(&QRic!fabCE~IWAy6VJuYpk^?{%KqeC1u7Jc_ndT-vq=+QG* z(iHB@=Yop|3&s1WPs?ZeWtwc57W>nq=a`>IbJaxciLk}&2a|(#&jzUSDqe`F@&9T3 z1VMP0y9z=w!=v6&$EQQbknjS-kFRknjb&J|aCWdeo61yvK6iBGMkApYQ>U@LxxRu| z)|HGC-VHn!&Vk-__5X~C+Z#bAs_jW1OZeX1eQ!}3e%3Y!6@?!XV>p(p|Q zKnVE8mmW^yZr|rLMY%R0JHYP?RC|)7Zsd0c3~9^|Ex4t<+#>eKhRP8E%vS*rC--e>aAMkPeZ8$-Z@z(Ey87s?7 z0=|7!gKNQqCzy0rH%nT^g`<;`X`v95SxFt-qoh9XSXGK>vptYNL|2f1BXeWm!mFyy zn{96=F?At`OKVruBq(#=SZ8nc#)j{;8eDP*%g9o^{w$zR@H_JA)#i@byR!47wOyl@ zL#b5;OUwL?pnH{~ZRHkyW6J;vj%5m-_7U@1T@B+b?0M@~xncam!29}u!reHDNeL=1 zm$m7}ajZ`bjbhQ#rH_>qc(8gf+pAf#Z)PFV~ z^&E*A>PlW6#d!Rp(}-(J#H)ChfiJzH0J87R^+^ky+4PRRk$DGi768vbB=sX&M_=>% zoXoGJ!kQ@3RCngTjuj(#?X%4g#X^D(kmqX3iq+oS0H4I@fRnPf9kFWellwSmdRoEL ze45c~Lw%NlGj+#jT~t=UXnhfWw4pTz)6Ef$^}X6OMh?>yiob}uc^?H$YfhwFmUTXJ zkXiAzJbk<;7iLBM=Pngh-^qe+$w7E}rzm>ygVO*4w;w`o5gFex#05_is$YNwqzmA{ zGv~=WGBPqpv^TG}p^Lq7oJchw84=ZN9`X6Y|9~1ZgI<;LgZtK%_1L*Re?ZPO;I^z8~BUHY3Y+7 zQVNP?r{k^D0IR_v?YAy-ca-}BiW&IutzI<59~xZ|pNj7_==rkRPA8t~wPY;8H&soL zpX4KGH6r=p2kS&t{y(||wj$$=PdoKZwW(AYbP@gPGs}W-H)^^cX$bA&Vij2M;G4AT z`5LZiA;DVMcj1sPSuL@(HuSHM9fQB*?@G`pTH6;)K+|&isZcLHJ8%o2f1L}>o>UTeN`Hfw7$KK1%p8tlJ>XP@c{g?qBGh`48a(x zm{1^3a($)xyWe?-kzc{xPDn z@d9~h%Prd7a2YgHI^cqltbU3tDA8`_0~jIlX}PI^Yt)t-35tBTRn_l8i#JHd47RVj zI53{J(xTY=Hx0#PXXr|9%U_(HNc@TKVCd^zPVMG;n@Wnbw}kUX3h=ngM`ISOQ!d%5 zMw-67)aokWfEf>}V}?f!Tq&~Op5DT{11&Hs9o_ zv_!s~T?V52aw$Zf3C@a5~ceG89x+^Krg0 zA7@t};PZrbi_g`VNI-?vkfOBooIUrOa`RAXM+R{pSvk3Jq@a|Y>2#&_APU>Pq+ZBv zC%G>T`F>o(5LHc)K=K7@GFkeL!+3_Ftv+Kw)d8eyk!>u?t}nxx@&zXaM28Rd5|!zx zY>SPWU1Uy*FSxPJEjiymH@Dm$FEPO$*e6dn?HUEI6&WRju+W1_Y9SsHI=R5wyFG` ztTk%#$mSkHqbH$r+8*&nj^!gA$Ji$Li2k_t((H-%qtGB+=uMB3zP+h^|79rj@LWz$ zFTsOW(x=e_>$zE*TgOYL(d>a+pah5Ma4Z1M0io&#Ip^ndSd>ywQ`2plSdw{oOZOYHHZ{?&aAb{HgIqL)EYwMn-4nfBJS08&(H~ z-vOcQ3Aap12wQi$EaStBUYF?v%BYQ`rKB&|=wurFlXS90J?F`|JaqBL(Z1K6jtvPH zhx5M91_CvM3uGI$D}*~kf<0omXX&1FHJCpU`iY<1op0Wb|MK?>$@{rQoH4$#A4e!|)W{~yYgKk-MK%lwB9nDq!SAW6f!EjPvV zM3h)-u57l_ewZyfx0DRCQKWzCG?5x?$*_g*Vadu$N8DbGC^Km(S<02$x6h(NLhzuF z=bHDQzmMuolEXu_abkaV$&!-ls`DGPROusrh)@4s1zv8UB>tHDY-g@vllBb1gc56q zIpqy*eHYrPETva}nt=+!km#=qOPOwLBGMBZ2Z{0Wj7cw(|y)3*4TI%c@XIu13iphvlA-*fnC3|JR#fy ziD}kx%O3gDs7b0vUOFhh4kE4SD0 zHjR0jnwhx_rhhtI8_7|jK{yr&xY9NtFZ+G3UBUoZLOyOD;>1SRV651zC4Ld#F_NtW zH#4)IsV*7}y`{UFug`7cZgv`WAMx`8zy!;ymA&5kw!C!|O!C|d&;{5Y$A2GVwc{6U1Be0wr(-L7ZwqbN6db1A9#847TB{){1u=i$OG zd&k>vQSW!&JU`6On9OqC;I|GqOj}}ip=o)Y4fm8WUzpk?zqSRu}cxAMo(Cb%XXEsyrwMb`=n=n%Jt7Mv0JO5D!h$Xwmr$AQ91a@iMEli zPVQ_btB?*E)D@+^)b=)F$Loey+s!x+4vVxA{%Rl2w%M^brg15)Frux@xw+_jrLW^E z^`s>p8e;PmTN-W3dAxZ0(q9@Z% zigRhEZlv<(-m}zoK3%B4LS)gBhl&oUDXbVDXQmK^^s~Ka0bGC#mjmP!X0RAmQRhvb`wEgHR zd$}_|$m9)g+i+%tD9gpsMkYPg6TLM0Pz-&LAvlis$8NdRg!uSe`T!yVe-DssE-$?j z#TzaM2S5SYG=Ps78N{ok(-{(k6Kt=3T(h_$#COfk&W8CiagM5EFn`&eYtRK(%|-j^ z`a(;~p>~-|%X>mLJbZk7?d!?1&W-b_c=v2ZhnDmYr+Z5WfC!E|aendOnkB|xyQHd!$b_$CBLuo2B}G+ZpD31E43?BpPk8zI4t`09Sup|6w+&HY8tFf zJQx4;x31(dh$8KqzZLU}e+%5oMK{W}RD2#T7eDIx4-lQFxE`M@eT1rUW0BBIT|Mp# z+WdUb)8J+8dRpIS_0zXviF@&+p6i#fq`@ALf zI$MnwK{%{*lc7T5EW}CX)djk&YH9P{&^NK~5`05x-axi@?!~X09{%S3ZmicrHC;sY z4FaL_-CXI38L4-%jVtNh+B`Av33x4U-`z*TfEt;;xLCL;h7jLeA|&CJyy|k9;YR(Fz0^|Vm#=6XSV_xIwXxBA>IZ2=h*~Ij^YUUJdvQO~ zPT13rsSu}Hbaik$U-DG5SXsrIX|WjbixMp_DSUeG)oqYpU+lUJwozh@Ur#ly0%*-m z(>{b-qM%uwIa+OUFWzw^yRVf-RP1~&$H?+H|5H!?LupL{iUz%1$>{f8F^1gT0y6m z4M7ZYMDDRXd7`}Peb-gwgkkNC<+eQgLtVF2BW}(|({J&^j=!JJ*4XFu@o-0fv@$iU z4mCo2?2}I-%O&r%Ng>GC(PcB3)rzdBG1t}q90N7ZGgep;y-n0S_kx3iHg`4?4V(3JA7BFoN zWb+s|j|m@OgG54|Bs^Kt9G8P2h126htx95an4CGcOF5!cy9Qm^^P+f3`UV&D^HRQj znOH*EOKP{|J9syo9DV&zH<{gzkjD>y%Yq#k?)k=_+E|JbOLoQkfxbpLt5=#Gy3_Ts zOwYjK!us^A`te3gCgve1@``}gs_&)`17#~(SKQ++C!l{TxTHuEXH`iq`TVZJb@4Gy zXnGw!Cosp7Y1l2k`Excy@PU9_-iC*aJ!wV7$fP>qYly@1oL1;EDIHzWj{h~NB4f_p z|0?5((gDH3tFeG4*q>4!Ws&a0N#TR`t|U=G%G(MZC-ueYsYkutJ5!bSVv8lYP5){q zqd}V7XANlkD(WJrE+Fw=5eJ#bq)E?fwf8RS{dxG(Z_|el7uMoHK@GW#H>Sc`U+(t6 z45?!6jAYoH-!wPWyD-KG8(t*9Cm=8kD)>@cD})NU-lQ2lM{RO?37D<*CyGo6{YhNP z*UCML?H))HbQ!GL{L=ZS+1nEexi9KkILObPOt3puk(=VASLc-N@nOr1Sb|25qO#qc zc<=9T0@P%}pQgHt0Xsm`zm7!lef_y`7i)cgxK*0ygmW?N+ZTV6y~Q@02jJQ<)Hkc7 zN*Icpnm@%%C75H&WYlZ&s7iS&@!aS|Pj|O^q)-~Yi1Vu=Q7-}K!{n%_v~wRmY-yJU z@8f%y-U)nWL2uLmC{s@zsLaIi+L@`b?*qNYQSv#zMK2}ly{iZzzQ0Q#26`%yUBdvL z2^$`e%(4!M2I?c!8844#>|ugTQNt7-4goX9#-Va#4i2uumH<&89`vh;6VDj956md#x5@F9K<+zLHQS%$o zasAC7Ei*sSj{LCc_bxuh!-ogl|10MDA5e@S&p&Not8c{X{5Qww(Og7zeGc6YDEaVA zvb%R5CkfbTZifKG2t#~bSzGJ9IN3$TcF#4q=`?I;WXf#_KLnbDmfd{O3AvuVbAh6Q zb-Eow%QZi%^3^?1K#bX81IO7)@hc#K*(l?6_x5UTgX&CnLi$9%Tz1tlAadiO`TC#q zr&fcsalwFGF4#5J*3QZqIB#zp8`HK*x(2OZo_yt}>iJWb^CRoH!a_j{oyU)u`z#u* z)i$~?C&ms3ylHVHSU;*mS`tZ?-!lk1$)-qn!*T|WCEx#HOysxy(Q7|fuT$mO;ySQD zUmJPNMW#;*U!T6>r%dASSJUNMfoX_c6cMBqj2Wx4VTUFe3!5b zsZuEVxNfS1xA{!9oz3)Qi*L)(`tS=i;c+S-o>3qA^W*J;>HUsFMDYvnpC`Bb&*JREpR-2A;tq!{m#WjPmUxWfV}Y3c2A5=!w^n68y5Vvh<~-U`x{! zeDu5zB0w$fbJnqzUxCb=vDCG+2$B<^Kg{45t0d)B#}ji;|)dWZ7z;L?hFn8(V4T)no7$F9SynlTpD(_f>j=l{a5y16M@!^4pFpBWMScg)<>3>7wsO z2T*`xg=gxN7&SQ$rUP|F1i>LFI5CAn^~A8k%?uy2vMOnQ&pg(R1N0TB5Pd(mvJD$t zJ$fsErVY&u>Xh-z`;n2E`S|&BZMC=p6(+yt4Nk2&8MI{r6WTj_mQ3n~JAr7O6{Ql> z>|9)NHTDZg_J_Nz;SS)CjM9G)X($+L)Uc2e%sP}rz=!Er} znwo!U#}Gix9+YV%B>mFz8QozvQQ8uRn0EaMRzG64!L7stGyrw~M008b98A)8t%-n_ z&O4I>%MH$q|9Ns!FD@9IOu=XE{zcjQmtC^5|Ae_9;nS1NiJ=GS=H}*k@?N;SfqoCd zut!e^?<;rlZn^^I0_nIhl8q$mXOvp=yguJ7L5lY;^7-N2iC~Z{rf1(PBTe(LBAx}B z-0M+G=clq5e0+QyB;zuzVaipG%X0tDfD+?=QC}}5hNUbA#T^svd=R4rVhJ*6neRdP zs;-LO%FU_bo;mb<)A!Rz(Vk)acwpBr`!q+r{2m4%ZPamn!+I^@WoGiIb#`*JXBi3a zh}UySOYemNZQp@iy?;p?dyNLFM0Cf&2 znwF4Zozu&voVPU}ZR%k%Xt>UVT4xu#Ed|!LOn$ooCo35AnSM0|P}^VMC7+o>@g8DSD@)8Tchis4#%=qZw(|8Kw~lnrLOFQm->7tCxGX zX(l?*&why?F6Q4Jo`jO>{x{=vKOdL zR?p)tje++5`}wkzll10|4)iP6Bc_EZZ|kV0>{o^cx@u_|b+<)I;>Q_AweQNjc;TXI z4)Ak1;8etQ1S#UTAwIb`u(k8|PmFl52LSiF({v8+AEWDbu)#|c?Ct}p-)uABWgp!wW6D0-+`Pai}Ugs-w3{iPZML;+W zB7BgM+e@q&!?I!EFzt8=JCr)Ht4Z|uconMpDn|0nn|?+*{JQ-3aoPy^^;o<1FLw?` z)g*gv%VfiD^Qe;TYdnM6l*NO(l*aC1rAva6JES^oZ}6mQKF*Dq0aDN7gR-@?wFqK8 z$Q`hh95+)|srmBu2`w964y7i1PMySS_MIP8Os_7^SZvueWnH6E%G=J|eAk8Lhfwz5 zuE0$2Klk@51Dq3ekLyi0DCzLVPD?JVTQSLOC`BA@W@7{MC=CDntVW2$)$E>dhKI4s z!2uV->Sg=y_3oJyz}>QQa65*|5~sAJ$72zL+vwB(1`kc-z3oDCQ!n6F--y4~Z76@! zEIgV1II=~iVhr#Djy_abOAxp9Y@HbhNBfyblqt3`g36fiHg#u6+|SzJ7X9 zWF+%ThB0rk_3O8vJA(V^b+dp8KccLATf?J$U9c)-RW!_PI#276o0c)OBs(EXmf#j+0awxvt zu!;Aq+lTL3&!9sW zCHI!c#4=#+O{9JRN*zYLJGgT~FA<|b;95ufs%WDkLd)W$$M$^c8a;5JIEHMnJ*T5P z4%u+oWaqHZEmx(5kU_)jO&X`kLnVn#v*DT*L}X<>f=v8;bDcGdPZjdBQ`V*W`E(y$;$3r zmXk{D`eI{ad-N^=$+O(UrNiaXY7bDDTn3i_*P$61f^mH|`mcH$l{uZ=k`a=y~Z z6-U{Yog6(24S{vpGoypde!P3D#9B3%)f(eU<(h#;028H`xHWt7)O3e+|G~K*zU=)X7Q{y-gR-8>gTF+=AKdV)V$smN?kg#vf(`-GhYmt zvSGpM^$C3^7c`xE4HN5Zi~}7tit-1dUOFwVJf4|&2C6Rq>jSL5#@BcW+z-|9oRMP+ zpo)!Ew%~<^=w?z*(alC*Pg`ab&bp)+z9ZFW-BV;*B-zO_?>^1G_&~D55^h*2Prjoo zmaB#u5=m(V?Mbp*Ql`Adyt%x#FGdcjUW=smb;I`*deKOIE=2qCQANP$(Z(oX0iGLQ zU7aj&9zB`djtMHul!PTSXi%~}10;EY z0t1Vg6D`bzaGE!UGlRR{*QEoTGdJ`lC#OyzD}WhxI$To$vPk@kU}E8TP`Gy)pf zBWQ*HK81e&{)3HR++Sv!VndRumk)tQPcer;$Lr<>2Uwf@XWiTq`cVQ#VJ*mG8fiR-V>Y!>e*@- zHNT=-X#Y=sB^I@m!d#Ad*1fVU?Rsdai@to*;N16&@-q~up?W;Wg04)Z0LaGMV%+NA z@a6u*XHY6rOuA#tEV|IhvF5U8`(@L*+K+q6y|r1xc?t{+eQ3R)qUUx@>Yy`)JOqT5 zPUM2up-N^LYZPiJvr@>Z!y;gs@)1lHo zAO9iciIz5}sTLsjJWqC@O3cB51XM8p)=Dn>ErQZ<8AeG-WASSHh53TH{9N-EE!=5Q zo`pP2t`R-&v~Az(h&~bf?k@q2!i&2~uChIzee^1A>RN6_&2g)II=b^~bG5qnLYqfR z4t&uy@buiiF!-EGBalPavC8q`7`dz|2yEBUfrK$k?m&zkKOZQf$>)bM6R!(W8}&XM z3-*jL61r_s^LVRCyUughUZ_C1b-^j|kioDN- zu=U^9eED`VL=SMMA4I!B1O0^mtYI%P$_u76|KNKUTD{quWgKY`FWwXRxMlKWLap^K z#!kS1`-{km;#~T*A8;)HxvmVcsR!d-*(P}5Z(;H0EJs%&e>EHX?5Fn4D;ux$nz^RR zII(UWzy|;!Yevo=Y$H=zXNQaCB z?E!84CY6e8SA1aIcwEG{Qd+K7PhvxcWldh%W(ifEeZwvWF~}5)I+;-XwG0neusQ>i zB=PJxVkX;qVh2W-(mmn+nB0K}%RvJMOZrWX`SVTvZgTEO%4YT$DSdO8rZ=-bLC^~8 zYR%3CY%2PMpreGaQ_QM@j+FtT@g7o!=S2$YNo0B}ghTU&A&A6K}l z%anCPas)d;h5pb9|IXE;?K%;4U>k@Hi5<9g5Iil5R^ zWX?@QzM0Xa_>R~RXP(1Hs%>P?P2<^~GTjjcL#kF_D_|RS($&ir9__Mfp&6=7r(L_A z_t79LDC1v$xC%Ub^D|WZVxs2bBuzI-mOXh_DnS7E5;z6mkm2Z5_WNEhrH4i9?zhaY zRwPw!JCW3#9F_a9U>Ep>fS;c=AL zi%S7esImyu*J={1xGrV8Fe&wn1IM%q7-gY?r{sc$SZZ$La zNRwnl6%9HSu9O$dxv+i;*um%-BwdfR8_c?hQTLTWyB0kW>%$REbK*92kMxz5{XiVB zI-F0f5HxDy=_Q45zpYjl&#>b?H)4JG^l8QNiusZA3`YB-uuq>i@3wVMqpx)Ylfc7k zy8of@v3cxEs#HYNuYfQ5DWl>B{RKyR9^IUWf~JWtD| z=IZ8)rn?wxIedh%pey4N-lgw}%W|p4{-VI-_M^K`@Ool6vx=10>_dXa?>99ci|Cue zC$K`;7R7vA-bTo)lyNae@fvk*A1&Gv+XRC)6B{}*V(Kz|M|^JjE;$Q`6g|^X;=Zno z)(5hkbfCX`Nb>dTVIThIfbyGZ-Wi$ppY7zjeH9Zu=W*8ow_Pe#R#UA;kxwRwukRN_ z#YkO#iQ7h)Xnkr^Hng!}pxUoAXHFLvsXNAV`~H6#U!lN}0Pw;Gq7S^eXkxq^d6 zHPCUoeX1hu?&;Bpq!V>rq3e*9^+_8HM8k>Q{gWQhn>ijSMz5u%1w9N6=@#7o`QMH9 zWi_e$#c@#>ji9QUu(_30K^LD{XI@w5Q^&uu4}f1J#i`12fEsE`yFFc{72r(+dIR7> z1Jug8!LkDb+!_omh)mQv7}~b_p##Kuy1$Iv2e1fPR`%t~7x&qD(1qti!ZBNNT|9^S z`^SVQR9R^-<;apGeVbFr!R!KDI2wYW_SI8KR!n_r@im3~>~ueKFmxig^{7GSb}|7; z#`#a)wXYw`)_&$3x{RDYG{9PZ;gcCQL_mGR%_yFo% z4MBVH@VEtdK$YITO9!lFhd$mSB!g7yadG+^k~cM*jg4Zcsl89b8T-H;=5`MwAyNiy zvobI5)j)w-&Qe&uxtPb%D*^lYLVR{g3`a-Da;H^AD<;1Mn!r}z^<4#FZ)0S5ro&`X zR7FfHq;6*-^vdM-tqrzHwY~Q#ZcGq2mhvNiO!0@Gf#oBA2xm=Rb^x!cCWr&eVecuf z#x0SItG5+m8SoP@8gGJRO#Y-vb1bNX>RC*$f{uSWr_Pp6njm2JGi7EEcN&P8`CWjt z?hB_893%eb612Sogtza>$y;Gq)CUzrw@zExAYwWWv#to87c74(me8yi$y-0yq~oL0 zCOEVT-a_yC_TA8-mIY4$7aZB;RxC~l!YnAPU zHk04aVbJlxy*BF$?N5vPHvs7H<0qGT^fiDwWuoYxdz`Q9c zD~F}P@C>yAs%#C^=39JAZl&EZ=EAo%stBYm;2E5YY3qfGiH;=gaho)VX#|wRX}0_R zT&x2F3rc%VZ(P@}Sq};~!B0i?02TB3v){r=3J<bk7; zmvRi(ndU(vAk`3(ud5N-kG7 zKGfVd!ylaEtl>L{PKZIeD-L<|AiGD*%rES;{64nq>+^Q{e=Za6zYg5~r5)?KHMvBq zh34=FNHz5p7v?9Rse-;GK(}2AaIxN&pxLC@4dyOF$MyBX`B0O?Y5A;`+~nqWtJM0m zcmlS|Ba=WVk@qh%D|%oigXFdkwtz9UsxmY=fsW^TU?&W_>N7HFIi*^k!PyLjL;EN( zgiel6qpi{fwII} zaX51qgROOcPPvFQaOKu;Ej>^80rP%d2Vq|bx*3cEt8{KLZ7rXa7nc{D!PSfzGzkE+ z7*-?%oQoIt2VGaPpPo$WsnMT>{ro{G#)h$D@%s4Pt!9?hJ|)f}Z9S=m6_Vl+5NP#@ z-NgvOx;1+a;^WVgY>wb#=YcQy5?nm@!3@hJ>vspM>yrLq&@|pL2HO8hC;yKXq?WEzHPrc>5tVE=8|7z3sX?c(7A`Zw1Rue$C(A7ABH@H;7eyT&eUF*>hGh!X-H z-JFedgb7~10t64(a-~+u&=?cN`i0K^te&w*I`4YN`R+YkIHT?m8xq%#Pc&)OUT&Vf6$C&_{(^5p5REp*qqUSOe5%aj+GxnyOHG>qHOIOZ0O&blhA z$cbG)+4}bmjtBHK7(ZxTD96FEY|h0RZ?VYPuTyzEFD|W;Q*_9AVU$810DucnLcg99 z<1Aivk-MSl(mJ@biRG3nK1)!PBu}syH$h89@4ramq#DOMr9JJBiqTy$ z2r57~F|uDXqwb5^%CWs@68pcGiJu4umZTmkC$q{B_fd<_66Se(pwg?LdX5z8KNHs( z2F}D)&|P}K9Dy9AG%_-B6k7o1OcX|QR8ZQBw>FqZtgM^tyVK3y4Id^;rPbg(ml6QC z#(>m^6qapEZ}vyeIadN#mW|PQYf=Ny7TkjN^J9=Ler?{{#;wi+1xw*@+ZcJADE6s5 z1(sYB@ELF*iomL=bH29}kizK%T%Y=-CZAJm48~0uMMU1ng;QrA%&Y73-X?x)z}BXa zPt`$kZ$~KA;DA=nuLOeHe6x1}f)txX`$^(|g+hzK$5|l-1%>Fpe@(4{F4YJaULMDF zAjLbc|B=onW0rdbBs3Ixv{tUFLWAq}SUK=r!Fuy@P13-u#8dluF&*n)hZ;+qY;1p& z3_s!Hw70jL_r|h&D6ZUE#djY@wIObcwUH*@{q?zqnnO=QI+3wePs01WQs=;oB+-Jp zDk3FZ6!x2#Z`oa+Y8SsB5*Zl}+*ku=HR|j>`+o!91ew@^#L98nj%fC|s{biy)=9`u zGSqd_>Olt#iihH_ON@$)NZjeimu|5RB$8G*Bfb^~v#hKPDZF>Es#MiHZ~=r{m4O!T zi|X^j030ICR4X9&+DOR0e3}1g?b$mzT|qFhB*4wBx*AMGlk>yuqHW>XAq546#pUV# zNWK;;N)SwG&s6%8Uj%*j>|zQPEAk!6l3$XspAYENbyER zN2lk?>MT0oOsTr~qieHiw7o?0`{cegTUSxx^nnP+A&6G@F5w9$CLRvyQJUuF9k3BT zk@oW!GBVn~=ddvILYAND0Y7qo-Jtruyu}3mVPV|ft$7%~B|1Ky6^yV9YZ0G9Z-;|{ zG0NFmahd%-FSk)6+pWHI2E_cTYxTO{=N%&QFYm{#A00Wz525K8KHzP`83(cR_J~}g zg@EJtDb1@>m6O3y>n23^BciE$-?`YFV_QNpt|`0V+1r^cy&?RpJE(-)GOBX$?6xx~UWs-Uo}=_1ba= zMnPzDwY1gvj$?%dEzg$Mui)uvSZASr1?(=N*X60k)%av2q1WqHNa0~98}EN|HooS6 zrQ#5O%Ku7z8H~S805St6=vV*$H*ul><_}OGJYckPAze~mU3uR#%k3Pq{%7B_?)W%` znHkKyuT)fY&iwr`_%mu+-u+9|K1;waF#_y@;;Tq7BXh7CZwzx@YR4AWT>%dH13)cr zvIA-L?fFtL!wMJzdTkB3c%X&TfUEAA!Na1)ub;2Lu=L8e8@gO%|`OE}VeR z2qaZ78a(!u8;I^ro+rqWrLRSC0+_QlcaU=ycw_J%t?dm3yd#_^>?qciXxVjs2fE=U5B1;y#@y&H)2- zc4?lTWQ9%o1Vz?pp3VVl68=N-H8;8%IT>b(BTfwY_wVzlL%0M4UJtPcTO=kXf_2gr z$pA-t1EPEk;zcmM&%i~nbA3FMh2OI3`y)HF@3=46Kk|G^`)(>84#?chpx#x{S)ujK zc9H$FaD`TIYTlM(Nd=OW?iSDfV)gus-5<;5n`A=)(H>d~3i-4x_nC`=pA_v0E6cR< zz8@E;BRi35>Gd^{wQMk11Vp$EVNgl)xnhdgr;N3C2D#V-_wpQ}dy@%&cOqk7IVzc3$a>(F)2c$!+k5KLpw@F!O>dm8c`_kD zVLq`$szXxhSyYmhm1VA9zESE0w|En{Yck?{Eh2ZW{Dv99{c^fl-t6&~Ex3L)hCyFk z763-zcTY;BqNC%YU{!VgWaOQdmmoUgcJO9~Q_A z4+s+&(J;R=kxDU@TMf}y6@qeWV+63#)L>%P8i?d&cfczXK6omt7T0kJ8t0AtuO3J1 z=@3dbham5P6oP}BE3x)$%@g1s&pz{e`Q2f+%*WEC_ePJ6(=|EHw>tIj!Bx>6U9q)7 zQI5!z=8+P7uSxFLJ0ez|sa>f_k#SWQ-0N1FW_yXZ3FR^-*Q6Dn)wBN&U8M%Sn0sk2 zF0;YF4yY%$-t}9!GwTG9NX+c4f%vM49sNv&l^WDm44dJb`Pil5T@JVRile~q-81N& zo2sx=?lj-)KfrqXPZMejMQzW0i#q$hLJ_^=%br&pHx5E8IbP$Aux&T$hWC+;pH8GD zW4!L%=C=&FQcxRaOj@O4!ohHWlA~i8K7AM%!YyFYCtCXSy7*GUwaShn0y{}nQDe%A zc~{BvQZuJmh|M=It1Anf@{_j_UlbHJSg&9~(cgctbK&w-Un$!KaTA;tNPZ5vUC@-nO?ag!{fwHu z@;^=*+*%Q}G$x7-EdZ?RKMUCVlLCWtX9XZU5CERB?DHL(1_nhrUZ9GG9qYZG(ra6K1JMNeeOx zHX9ZeD88dw#mb~6^fh)fEVtH4Zw-Jl4F(8a1F5Ns2#1ud6(nv$fAq@j;w={ajG)&B znj8hLsWo+>Zn2fLm3rT`IFC7j0R?0~4L{hE`@G7$QBhHhh(X~&jC4ulA!4Y&i?nD$n~`LT`Q z`u&eTWvDV==QL8_9ve}%Pgw8Ed*U*{AGu}PjJB3>sJirmh-j<_NJ4{u){O=bK3 z%QnwbiWDMbNQPt{N|Gt_lsO@@2yOEwq>@YtnKNxN&-0LoqGX_u*MdQ=0Ya`}(A>UuKiF2e-31L5i6|MM!Q;@&=H?&Im@~Thf6d>KNO(&RDzc zo9cS+?{Xl~721>xcaK_$e%q)XUzt9(?Dw4tU^JRtcjOw@JI$*i`#XUblqs*|FNn_l zDs?z2fJdKFhK`&X9hg@BFevZ5a3J>x+6Xjp&zf;WY7PT$RQm`kAr67_itqD(F6s-vXdRwO23=`1c+i^FOUZlBP?6*L4j<3zHc52cKLr-a2^D6RZ7Y4`O$ zz9@4AbIHxqPBOXO0%v}O$iqAFw3-LxVA68iK{#-K^I6cQ$?kAPtf%_o!o-$NAy--? z$>y;<#!nG`HGRZF;A_+I(Yn9NbukhQVW9OKVxCw%65L>+G)ub#+n)5&YxuO;4@0t)Jn9T{Ln4%Lr}c*kaqJ_j4nNS z>LwC@fjv~0J$lEC>-d9=jBH28C3l7J-i+DFe%vYWy~1gc6l4>APF%}f!H99YuVnG7 zRloa>6#w&jYtqy9SL+$ieRq#fmxT>ta1FKTA~jR(~nA<;5cO zLi=Ch-5w=9TjT%or2w(^ncPb#9g;mcEKN8;>LXh@h0}$5hg&@yKmqnp$gDNX0o~QN z-}JM(igshew!&w3wX)ZFILowen*zcrjl%FEG$QyZB!oE@ZJb$MHAiW^-kXGb zaN0ZYF2!bJvx|pKm(=;?X_RglHtyPKUusND+j)iRRTW zhyk+VC%693ke#k84N>%=qj^)Q@)0a=rLXz!wGEc6Xc*Z{6#;otjrOmsyF&cZn+}c? zU0Zw_}jpf!RdohDd*~c6i-!flaf@*%sL}BMR(*@6eqvfpR5p<>Tw8ImO${ zgX-Wf+tCq!)EP*3R`zfsD0xO~+M*M2{F(M-%6&8s@G*7&$XN996OLFZY%jLba~n%0 zHDgtk+7%{CYvj~Ynf22_vmcL4HYKiTH;$aZ&MeksX#)MfQPQW11kg_NnbsN1X+A*s zF)5nJE9gHPX=V0K%g!>_^-)3ym8qt(5#BImGfBD!zDpL7fo<8JA*KY3tbHN ziQ@)5vtG|oNj>;@$%N-yea7C{I7Tc-1ijRp>ip6bQ}rUcV z+dF}NA~XcNV|kWm$Qi`QWGq_nV-*cWeg3#8sj9|6mw`n$=aiUF0dm?4oQw3_S#tCE zoe+OEp1%**_4XHck|c72E)bn(_kXSbiv3r_ovyUyjIKbuc3;ZmUQh#c^}pNMy!2en z<0(qhs=4U}qww3^9PgQNPgYKIWW3;^GUL{(_a8uEvmS% zWpl2ppS_?&73>OgN5PZ$cmmd+8R*Gfz6z2O?gcw@{Q71n?%|g1ki+I;z|yP7&sFgH zMgG(h!KEX$W>dyWfvnYyxrFu=O!34&srgc$V4q zI>t)bt+sO&gf?fjVs9lG^Nqhe|Ei#!65F>rz_TZ*O@>8h2Za2ncVyseDdo%dBNj9SC#cF+5 zM?@F_#7VX+gob*FM&A|wxeU+Bk1t_2XTM7k;QfqZ(x=E@ncpKL#5<+2Hecr{u3aZP z7uFU3gGB#4y`p>8AG;E4*|6^X;bjVvaL?RR&S5<+xu+(*etoPqb*g`SIV_^*QvvUh zL!l6>Be5;l!a83}d!f^^2$R%TllQQN3KRrXUnH-tuEz4}T^y@&qK}Nf9F46y`)l-9 zB2U04?n`}zo9YiDObp?sE$J?BJhB8X0BO5?aA0$=6)Tn4zUNcUyV`n%qe|ylC5M!2 z?x~rTqTEy7%c>7kl|R*8B4SsJB_R5(7W4Jd;{ENbO8#j1Mm7PRYd;xyPLQ$;{ldWE zJ&yO%T9MEdS)Z8x%HDT7#fcT8!nA>5xis9gM~CHpvFi`12nC|ttgW+_3W}iN^(x>c z#EYF24miO7T+`Yg1Jp#GR4SPoHhJan#)2RVcN*XM+LJy*e%4iPl>s`1mz(V0c)yV1 ziI4MEZfe_!I^>2>Qm0yj%kOw6nhaZIt> z!W|`bsyZj$jO6-)&6UHcuaCnXM_sPF{6RFcvFZX>dbXaIIv(m7QBP}^pZrPHx5k?0 z0Lg6^J(jPZe>#kiEqWh&WF9D23KzS+%a4lI*U2{Cy^~)~py+ zOU<5CZ?DOJUTkH%v-&7*r#z2uy!h=Up)ekO<9(&~qWO)2R?XU>bmfyDA;;>M74GoM z5=TZZ-00HD#l{;Hm}WV=)(3tb%{ZwVE^v>j2(lWhPbAdmjT`N2zbEz`j9nIw-;rH9IOYdfzl zBda-pu^&C%5?tbc;upK?Nx+ebMTM2Vd!E4}R`7ti?Bskx%+W0~-(SHQo~eZmQR|9# zV$CX2qMeP(V>uNwWrUIpvbY8FF%<*A{%*TPx8ee4M&|(kpwMecjpVR%Xgsr;11Qu_0%wcW1Q@l}F{8 zFOsi$@E#SNI=!&0uc=mlh3|DexB9!TT!zu3(UfgA`ep0BQwdZ~8VHd=Iebs>rJ1)E^HIn}rSCx-@^~*hjze;7ad{ zSJROxu(B>i-RXOpJV&3CXAL_`9%m~v-j>}vYb*Ej#wPM^Lindyo-rLS+51`w$nk#- zL0cM!T$TM%oj$qrQ_Vfd&hn~a>eSABt9JF7hW1Y8eG?PDXCtm3cA6lFSa71?M`>+{*-S8mrT^FlHIlg zI+|hbql>h0J9;9aH_X4SzL>e+8O!zii_%dfoK3pn!qm*n{KLb;OSgEIe?C8Fle$l; ztlVeFJGMPzDwgo1#UC=M=uc7aCGHX!(xRP>3^l&?Uo#NkmO^LLeeq8-Qz-w%ww{q+ zo5X)P!b>K3ibp>lr}=08Xda`Z;8NJ1Ug zRNWBiG&af2m0?qL$QfD?QE{|d7wKWT$g3YUpgi(?*{adeMYJbhO#p|1-UC_%Fhu7A zHY8UhPsQR9>l1y;Sozg*ym)_28t93;Y7fu6CbcWMC{6iLPxYbp&YtA+x3Rt7i%St7 z@~AS0c0UrZ7N>kXY_!!m@QWtSoKUnzJ>Lo@srEIk!K`fTD;eYXDWy+hYwkK9nlCi6HK_T%6xj;OXTnw!DRS~K@bMOVw-j4w6rut z<14a0Xy&iG1#TTy$ZpZpr}sPjr6>}CV)_eRZ%XFm%lNsgz&FbQ4%kYH(^zHLPdn#6 z=+dSg^@Fab8cC8n&PQU$MsYgvW6wTFg?RWhFT4P)rX#-U;E6&_|u~-yi39v$bKa>}zFjYpd`6He+OG;xDVDILo6_<^03^&&6u$^6Tj{ z^0UYFL>2zeb1mq2_K7IZxx|M?4_2k1b~II4q`h>0uTQRTzMcP)n1|CWQJ6-X@VNh! z=|XgKfV-UFE>0@Id;pNAr;F4!DM2--QJ+bsC z40a{xn{MiOGhogI%?5wG5-Y7wG=xvyuR_ez5;f4V8rnX)w|xI-?NrWRD4Hfxdtmgc zlDD5py7q2fWR>WXvQmn(*u<~oOOHPD(dS4#KFxo{b$zMvW<$U~sCYlq)gZ6m5}Hog zEV1^Um7QTfZ7Z(JtMw-51@+Ot7%FSsloB1yA_q?bvVH3@C>FKn*&NdeS>beaV9`p(}uj@s11xXA?0eaVp8B@Y(lK@t3i z1DT_PlZ;+LKOOo4`7?K>r>DEu6d<5vdF5{3$lN95C1`Y0dW;#1m>!YOkQppk1PDqM z@&~=#!w!`4D}mS}{>{HXK2qVsv>Em;$359Wxc{s)sOdk>(d46{>1JcY%0SbLd{0kv zEdCqD1>+^;f9$g}1hO3NOP4gzmVG&zS_TF=*(->72wL@>7`(NJKcE{+CY1uXIIhz9 zJUcOf2zt_Yq2;`@AjJeiw(i*08uCW~ug3zQt#a#DNb*e4qeZPxot?efzbTIpzUVN=~6}IW_t8Ur`Ra3;|FJM2SqN38X zSH|nm+N%qCKtsI-2Z%KN(1rRF+1VVl^@6;`-s~qjT}+#h)B++jW90E3`bvGOiiiP&lH~B{ z2Fb^=I3+~MmUyh*jTNOohP&Dcerp|Oldpk>*MKRGdk&*smV&;d9O*q$U9# zgsi-Rx*v*a3-UTL8bN`;FDt>K&jeQ9>rOSmCjI4qAUZgy-LrH( zZG{2kG*Z6%I>_gN?j_^3MQy9#l~+N5hz8H{+5pQ-XEA=pp&n{-L;wKWRrY+mCH>Or zoWwWgI1z{;zRRVAYFI_h=3{a08VF`OPA0d5Wt*BYIW~-8(}*HRW7z zks+1Ied5W#Zj8K#3U&rwyt5alcX|Nff7$U%X{PLjYJ;2dL%*7Oh z&p-pXXxt_|;?dj$>%2e1qJSLWjsaF2g84uKvTYsnrs5F?Z9ORPd1?d&3rtSyohD~6 zh#+>w!{sprKx0t*C7o_RBq~2^3edhBsWZsYNdO-117` zRsUILoT)eAXin!q?pkx4b}UE>bJ4e~ngcMqOAz3Zyh-sdm-_$2)BfMGy_+Pk<|2_N zAh+}HX43iIq*iYM^&QsuMaUUY7imy^wh!YJCNNjmS(teEcZEYXrb&6d0~*y7E)w2( z+tf61q0_N@QO}C!3?y3co30o7s{AO z*v4|_UPVV|_~pSCCvIjlpCs))z{rYpP>%BEJU*b_)azjqcYT)-!YFLr0mc_jM2FUB zfBW{+govJm_;~Kz++5eCKJK0PqSYzqDd8EnWO27!PLmCn@d(K^3UM9u0gB+UkOUi1 zmFW+E(5_QRKTZlYH~n>A*PR}{{E}wgjnR6v zX|iP!32{rfr9eLF_cyFG7X7jo78BOp4LEuc-@`u-%Vo`(@aV|jWoCXtocK}>if|m+ zNdg5JA#u$j$zt66q*M zaR_IRdHovO^J3Q*4h64ctZrW4LW|6-vuWMqY2!&f;K{TbgNJw31Ogm~9OxiWehK|1 z)IDIqGV!Cw^VKt5=7_Lzx5ew~sJVwn#obVA@9W}$s|?GG?c#@bhOVg@5-BJsEO|C> zeXhB-Rh07Ph^p~BtO#A^{N^Skl54U_ka6;SBZo|-xHZ$ChB$r`SU4N;F8N2>i_zO_ z<3@;Ha2ZL)_hnge9y+7(vIX$+{>dTGbtMqSME~i&BsvYdtQ|{BL%1VJyKDX=1U%%- z2o0RyECOBhP9@GV-DE@{T@WtIRjUO@TbZMD{qKd}`8$_)S^5tW&CP-1TapXLBjGV2 zQqe?IX>Je;uC@B}u9LU!8U?Y#6EU|hnp)sBMG)Qc!Uf^iDmB?K~n56K!RTDmr z*v|_Q3-P6gIfv6zp{RbZc@_c^%D46d`QO}@);r}4dhZTDPV+Tfnkp|#TRAdqK&MWy zEYs7Q@s{%A1he}w>D&j5Ee9ul5(mJE^$GDUf%_fQJH*x#Kn&_$4+*>gKk zjSn46n4~`3X4#Oo5N&<|q#?f^@6ZQ1C`3!2!oa>*v;3+*wUV zq3akZIKhcv?3KT#v!HKczcAGjmWu6MA7%n)#tbs!kxSlKS1nPY?>A?8VNwRi= z80kN&4_B_z`yBFv3{V^Wwo2gOiHNzdKJLoA?rHL=-B#yuM9dvNbv9+vM7J4-^}#LX zW$GT0QTE>tx2hqUuix{-c+_F|sa=WDA^Wj(r+t13 zY-PDKBERWKP00xZU+r|_@GM{Idnnh2i?T8JEYx)U^mk(X&)3i528ds}kSgGEi=a@E zfRgC^+spE8pLO^?$Knc8h-u7BJ=tx({`s5%MI9IA53$=n%Efq}1TnlpUvl^?<^KeQ zzszW(^~(g;Ccnt<;c4dlFHbaZKX_dWR&s=;P_ePQ(U8Mx7=)4W-Oo|Pqi2$8f7=Xv zLYDBKmYQ}cozGvSmSb^sLa5~uc=5#WtW)oWt~-+YoGSREDwg+r^_w(9llS!QOOV(T z;sv;0wIAS%C33UE34SXppNTgbu=i!7_6HV+_P3z`9O#X2BHV3wctNtcA+_?H7)PTR z2c%V_@(KfTRs%XE8xwyVe>H%<(T+KgzSJa@RvVT`Cu#EYXBPt%<)$n>t(xfGX13Xg z>Jw_%eyC~n5zr^ZnWXTi+H!e;brc9-9eHa(;6~r{wvFIz`N#EpJT^m(Z~_(Z#EYe2 z7G_G{;G{MLVfI5T#Tw}R$lQ8SxEK7$j4cT5g)bc5dCWg=F$%y$zzogA3`C^!P?r

7NO5#Ho?FIt?GPcbR*XZVX^AwLY93#Mk=;0ZRF<9__#v?hP0 zvj0bVvAKVXm%nY!|MnjTA7P(pfjxaKQF6q^{LfE6N7_F!{l6%~_|+!Z3duIWz$t?ujxvKJua+ymqw2`E`G8l0H{4nTK@noOyOCNw$<)K2LRX z(MhA+f_1aO!9mY^|GcKDX0#kmtWQz5JgbK8++@O-nc3j%vESNv5j7EC=?2LGOt!5v z7a@3$5S{Y`IsSw{UnCCPK>Oe42^U%i~@)av>rA=Uo5-Hk3~ zKG$ksVaD0WA{qU`o7FJIx!?h7e9vNXlg~>=DN>?0+XFiXLGYA^GZI*{ngvm zwIpJ23?N~B4NZrWde0HW%{!_R23ruI(`{>y+O^dwifp&XLDTsdq}Xf@;L}h|K!yd&@MQPKvr zyD-GDYn7a($*7n)Q77Boo!Dcy>$7WHbl|f_?+gEWCmC9A_N)Epwk~Zw3&{1+_u!mR zO;oJaiH1LYpHfk*g1WC zemvWL9}!>w``w9<;A0cVdY;X28riqS=l=emZ-9>ydrmpPbpK1LrK-Eie|=tdYIAHO z^1TFhBK{-DxAN-jhiCL1)M_BmHmUd0{l_?b7gVOOU*1+H2zm2{fl=D4EP1m4IEUL% zmu~AENrIOUN$^5|dg;(hVgl3~g~-buPAm@=BBdxTEp7V3;{j?+6LhKXKv3KZVHP$L zeuq_eufZl&UdOXTL(FH^U|i$AoR!@hM^v9dqz7@ zTY#P_Bd*U_ODhVRbZvDSf-{7u<5*Y>k&~lPUtq6#z|-9GD?bf|ff!*RpL1BVGBX#s zdzRR#n>-a2Ji$`$^)jfC5CuI<+=mZ0A^${|07V=H#cKZM5amdv^A%8OX@3_9UbTma z6t8EOs#5QLn*IAY?v?u;!MqLTZVy@XkDpOrqCZmNWAkKCxW?gCkokYO37LHK{$>>T zb-A1voF}27D&SPw*7+g;kF`IBApoA}z4svXWkl3rFjFF{#c&J?^c-t>F}jHgMT`O-(eHZK4w1I zkdl_W7cgl7C;F1`)cUk%i_H>538=mm(?oL0{G1PE=1ZsQRkw;_VopIf@B)c?@GXSK zAs1~x^(J&-rrvXPB=ficO?CJ>1U&(k#D4G8_-H|gApsFbKc>S~5ZKl0^sSpuln*Md#rag~Aj!*oxg#kHpSSdV=-~_@7kqpQoqG(?# z`)D?)9(sD~iT3fpvc-Z{|~ML&r|ywzdQ#4Y5i z?3VG{#0_KE_XmD_KKKtQEpkok`aXh#plX*7}_uTeC zth{rIhJY5fo!LQ+>m|w4Hx&e+OPvz2KQUAGbOv? zGY&L0VQcj9H5S_#4RtO?YUEzEdH)`R(|x!_+9m}Ku4*aSig--ITLNiI;8e^Jy3n8a_CJ|>hA=4s>Hxc+s< z&$3vAx~U>yx2$DXlpd|K|B#@WbnCM&zkp~f)6o1_rMAw}uOv-~Oy?cSDq>*(%hzk9 ztCgxC8Z(=rJRmNhe6#b+5!K8`L%@vOok|!sp+nH7Go||1kh!)D7dDWNvdQMe1Yiny zG-)dI4)gOdIH|PZjgzmCJ7HOW=Cnk}XnTL)5+Ulrk3ivFLY{`KuvgGBq1M!b$njAR zDCdB8C<*;6e=w#v1 zA{=o_l*6IdvyVQllO-j>k{cn1^Cb`*23ebHZ~+OKQ-qWG`FtW2KY#SDSzC-%26R@O zhuv5IPLYhpr>mHzk#c*JH5M1=vpR!v<^Xy zRH^Wtb$MOxCgwS^KXPc(R9+(!(%2KUDc39(5mip#Bj@qqq;B#~4YVq_=rB_DY0d$w z>x+*HAeENE<0*T5HTLRmV(`QHfqiM0pe2BR`-R4cpL?^o2=gWs8{iT0!l%)&XtU23 zC7Y@LEYWOpLzpe20E6RqCf^btrz#;x*~J}&n@_eC|4J1nm^tI~e(OQPv&%diZ?1t~ zCV%B~L$(~w*J~{6!Ruj;WGRwhc)6U%Lyf~%>37=R`=Evp6@Wt``e3GXy0Vl|HI@Kn zp!nEqtGbAv;?UQ@nK)}>_r_WY(b|rb&(M>Lws#Dp4qvOa%`ppea(gX%=5tKsC>TZ95|-JQpH#fA3Ua;b2F zAU2FW04YFC3_~Fj7BXcC1miTCQ#TAFNRMI1b-=?w!>9rPR5DK-sCT*U zynO}MV(_5>rwgZ_);c$Mey;&@5z7MUS_y$sn45wVx@;-1EcGczFufAofoZ0cQ5BSH~Au>Y;e}Vh}rSnhE@?Wc%2jgv6d9HyE zl(KzpD7=4`G@IxZ)A~pYH&0}y&T15j{-#lzfC%&CpA|!p z+1smLV!oxcr~O@%p63|JhUg)iZAnNbc{}cP;y>H{+r-L5&e79XV0+RTkjM#vC+#JLM!{*3hHGQ%J40!mmp4) z=b-*gpfbK+d-)MV^$jhS`ma3~|5^Hb`7?3%K(fqxZSjxMaNYAkx}DN<0^`0d5~HO} zx8F?qeIOM-?_ty!c*wZ_d-5Tl@$u08`@xg@vi|4Ux2`b{_FPo|`=~{#52K}w%f~$m z$9%uLhm2KeA2Er$P5QmsV^Z%pDNELvu2*d-;TXNs^<2MtUl#)mi@`BW&rt!Qd*`jI z0a$>9rRVJ6i9+lh(sSOf6LoQ*PVP!^@awuFp&nPN&p6wEGlK~AbRLS}Lgw+g+1cBn zl;;t-y%vmnnumE@pn?Z;?sueLoK#j;wi<8H*gNrKp4KXsw^|>>CDV?t`rPkxkPGc07ks=3$LsN-OIuh+~!@e&iIaf~N=4=yAP4j_Fp zBySQKFObYD1l40)`$sLUyCCIHij%!o-1yD&%LFxCN#3nSHXIL3Qxu2F#Vc3T)x`m+ zZ%13PiipJO6TlO2c)Pisjj`U`Gx*xL7}=!f(k=AP{(C>`8E$^=$e;zMj-L938lPLc zS096S$4ykeu~QUHik~keENlgd37ar}NF17|gh2*OQ|o8GMPaLq)f%S}XUxqHC5+y_ z8^dxqOct)LCBi2FVeZt85Tb^feo6l&U+bbFU^f%UC|VktX`vH1`GjB=ntKI}Ex`D$ zC&#|xkOA)l5^?MoS71cD?BAe7$7`Zl$J-0BE_j6?_BQ2xNTO6xxE1@TkuQlP{FN!) zz)*79wZ*PG-|z3e&WO5OaWq=5Zh5gU1KCe?ZA!o#a^$%=`>ES@B@eLlSjY>hk?I*; zTpRZ;@mNrb*mWk4uhzWh)IQqq`jbpQq3ObwK7LDUD-BBiq1xKk13T7xA1OAjJ8GxN(WX!?`zg-J({CMdOd@6CQ@i(A zIh-1gRjeBupt>=T^zbuxD(K)Mbj%l#lLi#V81V4mZ3FY(ip+HAY;SMxf}=pUK`>Z> zqf5WBCr9IxeDLmvHijcU*HF^UJ^Nov*noo?VG?}`FSWZ@EJTm;7Po5aP>kaK*{l7p z{L2zA!dAa6{AmzrhSTrDS(GzHbzVYe?#$)8f5TjO5P)_M9H+|0=XmjRHuOcQ#UA~9 zWp|p);H7nO`1FGZWlJ|o?{PT08fNx?D<3!??z(TEYJ_!HGm-dV!^-|ZB=Ycn#eY0f9~l0AzyQL=x} zlD+^e9r08rLhZ~!Ddh_CT#nVe^NhbxtA6vR{$r}{rHtID@yWyD!E>e*q&RkD8Y{WmP zyY+W_{$=duwgcv?*&X}!37uza=N6pAOw(*hN4z{4if@?SZ~WGC(F`-`aCEGpyH|N^ zJ%Btf_dA~jVY#vsZWP`+SrTab?c2^jGUKKeljDTkpjdH$WzPQTj0P@KBJ{^()x(TZ zf2Zv=Py=Po)9nR;fyOta`re#>)ApCBB>Tx#-aY=Z!F6e?pw%xmoaKx)sxL>!K8H8m z8>q>s&_9Rde-69#9_XL#@Psd&RDgko7O$<&*Irgfw#Myd@W9IM&+oqA4&hH2I5n%M&t8q7RWI zDs`H@*EUz&GPmC7R%9Hc`+L77$(jl%P`PrGbBOX!NW;-Wz9c6B3z$XOp#Z?`&}J=i zU)C%CKJyJ9^>gsNQx#UCN#M!A*@w4r}Y{7uga5{~Lupx%L(PESzr*)XVJCW%gi`bTy|4XZJn&*M93`BK%OMxF9)?FGj8f-gCC zGkbZ)x+IFE&y}NMYkTrY_iKF9daYK@>ONc-d{}>&cXWTWXU2YU>(2Lgp(;%@SiF~8 z{J8CTgLAlw4-FYZ1K$1k*^u!L%tTAp=4@=vzZT_i9OEKY<;?3z1Dp;Z^Fb&1=~JE? z3U1@a8jI5Cpu>g2g4jMREl=`gqRX!r^QgKOc`HqL@{TFh5>%Tom^T`Nm1JTvTLLw9 z@*IbRNs8~aQsSQG$oz3&H5Y*b#+@S-1TDy}(Rs>;FY4(~ZONli#ziqZt6KKCYg93u^WVsSzq#+2 z?01GOeSbzDgL6yP_hjN>^c%40EyGzG1xFS5&tcHDGaB+Hl4%V=afFN}gU9NM4t`H? zdv4mvD7hh)=jp&RAKgW&#uqyDYC2Nv&XZRpKQ4)Kva$7^6`}6v?)JELa{XiuU`FUc z`C%sW!)p{mTj6jPRb4%k-t=VB2M|iTR9S%`V#f-WeWeoo5>3B!d_tlR+K;z_!m@GOpj*v+?F;sEua|$bYbib; zGC8-i6R|%}!8KGJj`*h?=H~jgP4N2;qZ3}3mFSz$cXZF0qv7gx8X1VMQsd)!U;^lQ zieOW-do%{9=yG*)t_0pQTqjl-6#SyoQmz;<Y%bhg-&)ym1f3@(CT1=smb z$M~|t0H0C!*N&MJ+WVR7KJn#iImQj~7x(G}C;&Ye;Oe3B*$9jc2zTd!`?iTZZic1h3-}nABZ9@ z2U&kf$VJ1P$pN>(X!v$L*2Z#~xzrOmGP3w%ep}J+_B8An9*^CBdYJjid~c!AC{}k# zdbe|{@jX|^o8a2Hj^In^_^n&@XArPPz_w40RZ`LjzzQxnILavSp(oV|_HPa(usf@E zUsZDdZ=ZPTO>%?J`-Jt+<1YF6=@Hk|91_1QhT?^du+yURPNC9J)9(y&we!sPPuZm@Doefz!% zxE`P8Ncdc%hH1o@ntM2aS8mxc5hLnBU&E{rB`*0EtK*s=B+g)Oes{5geh=FlA#L+% zijm1`uENrF>Bf_>M(2~t|7dChrIFq#8iV6>_@N$$6B*H9LL=vP;Xj91cF4~BI8j)g zo7uoBS!{Cjz^LAjJY})ON%yh4hSke;Hz#Vi+Bja~M8F?BJsZ+PfC7atw%6IJfOv|h z8JZg%W=x=o8ymLP=o6JL%f4@xQ`E_zU7e#sL&+1rT}fous7!wSb(v&yX-D#?8y4x! zSzaGD>eP_v{XfCyNW!9Q^C*=LXu2GEH+3_6B<_F+q9UhaZ*YBVa9tJo_2VZPxqvVw z)o4gbxLx>(X%?akM)YQRRes7hb`6P?_$(M!dn_lPDy}?FN{i@vkx)HueTJDC5M;*z z*(j|DBg4b6&&|m972(|@7>NHw8YwMeKXp&V8_jifm)GJ}e)*b-H2LCwg16Req`t4K zo*J|M{3eZO4}_jzUM)vd!T@OMA)`(=fvs_yNBpUPKC%3Qv+c>a>gzel9;F^7CI{YM zx|v9+R`b$`o!2u(U(G{V1x=+;w7(#G!VN06Z`yM%A4kc}ZOmuw-mW5eTi`z7;b5Ve zz9e;%oPU)KNr748+PSHhK+~rPP(QfOe)>!0`@UK(@~j>C$qT){w`AkXa%=M?`jrsJ zZJDk7ZKLX#q8rTh&tlcBPCBMRxoP*#X-w7m3^BulZr48>7O^KlDE70zi7X7#)W!VG zEKj1xNg%uM_=Zax(#IXr;9+ig2ujrDr^_lIfAgy#2VW>MV`!nyT*r_#;Gv4tWy|j%J=oMI$M49VCU-|N z{&*_y&AFE6%g4=^EZ!p;cCF5^Tf3)~M|q;RMRU#pOo10XPb=Nhd*QQx>YS*oK0Wba z&&gqgo9atM+UXv$!wB<8=Ly(%n7xDK^~}XbJTf!;=j-^IOPfON&(5$0rzY% zS^E(@tk^~P)m&R`X2fUvCA$fZ6dx@BiuoX+QKYem44?dMa|29ZO{PWf@SD6YV7VeJ zT7h*bZ<1bZyrzoyBC65x`~4W;7p93a#c6GF#7i4NN!&XRoXig@GMg#~{RG{K0pVGrK7! zkz1aphY9fkyCVXZVyv{Rv9dCnB=#wm$mI8L$Zi9MGQ#X<_|n%jc{r6}&ULWxM6)%b zeWb((eIyJ;G_k_SUB8a8qqGa={h7SRo!SJfhfjhC_0wn1q5z)BmEzxNKyTfed%v_i z6>&i9l*tkaS-3-l1OsJ9uIgcRl_n5%-wzIGS1|ywjB6e#`JEhj!Z!vj)1z(+nBgb{ zRHHQgTewe%binOpUK=qv>l?gF!1h=@aB#>^K#oZ8`~a)W1K!wP?Nj7urK#n%!~A-F z{iIxtfB1tFMx_w!nPBRt0sHe}LM<`C@)J zTb;%IJ9X-b6T~w4N5v)GNvM7 zYAPz)R*c<$W*TrmD^`XBX_LQkR;m&hn@!l9NW0)9tV;wF6E9@Thf6RzR-Us(Bm*B| zOpVBWEJa;+HXg2=Fe|x0uQRA}K$ zLw_Nqk3wdqE7&~4dR*VxtCw2o|F9fRn!Vj+sb#eidyz3)dqG>UcUI$mlJ>e=<(%rs z^ODB(y~hf0lu>Xc(GWill;HK^FDpr0pz`6TZUVL8Xecx7=VAZSu@TjdF;Tfwe59Q` z`{11JEKa1$JICPY3tRl#s=JUXzMWT375hs}#Y&2;G+(HI_s8w7bQ9j0^Xj)o^$L!} zkHC1YTKk$!Gtsz>S$kzhTS(C%uDbKosIex1PlGKNGy zI_dhyZF^5EbyqaCtHe(A#h*9?+{>+s^&giw=M@S*KdZ|_;nem7_Z@b)8!{&H^lYJO zJuew2wdsxkQaenDBHV9O#3iY6c$rt-?Dw6QX0M_Xh5KTl(Rx+kx91>L9&)E z!NbQLE2fgUwG^$-Z}ahtqO^;iLF5Q?M9F)(eIu*}^%=-SG0N*_!vk z#}iw`@cTTZ#S1E$4lyr1*Dkw@1|j_5`Ya0rv4-3qgNls1yfgZpJTqB0o|SFP`^I%X zPP7zt6eaeSixxQ3!}gn@$lwbRh_IiEj5`UL;fSem_9nJo(_WQ3`a=)W;ADa}F7 z;=*Td*YzE5LyglfMjT~1MJ}sUeGZUq0U4DP67q(Z-~4L5q_275L z72S`%Do&qcGsel5{4m1sw^*F>24* zvt4CZ2{C&SXGS^V{QY*)9fhv$lGR-ki()YN7Zaiok3pJzDE1IS3w1o-tUXppUBYdn zHcpVkQKN;DCu`Fn1=2`#&{*TeY<1WHnBr#qK5rSvO|-p22Gi*F>-j<(;kC2Q(Bj1B#xe5oF^N2D31z}1Ed7Jhd;OR*@5x= z>z~C*BJ&ZHDzBK-(bw_?$NW8XUrwAeQ*u^dz;_IMSO2uHa6=YkdZ~a|*qz*TOCP~U zt!WHT`ip8y==Rpq28J}JdCuN5q@E`8&AJS#ab$?kh$&6e!)0gxgT$Jsd#nLBiUO>D z?p-$Mzep`NH;J#0aMIk`Svhru9alUYUxz(m9lSgIQ7N9AW^R`*ep{PEbezc5?%{Sw zR~YX*h(zFDCjWGVe_G2ozmJ%?<@AQ+oESfR-5rih*H!Z zQgLLnZa4{0;G+q|U1yH@hrMzW)ilht6cJbL`li{E{$2$XI=9>t2>bOuj#6RKIHWqY zNBATMoq;(AQ5)EY{4@FkSBdHUuG$FJc&-d~Idn$4bW=HvIVwD?1FmvQ8UDWObw*yoxPo$=ca*B@w$j0R)!DUB?8-$`cSu3_CRDsqInp(MJ3J7+wXZwW}h(Os7_y+b3D+&KeGzdp2Dm)IFBx&Q)Ezp(S)B_NcpIu&+&;`X_|I z+NC%S5uk<&iD2kn_E|zg^;U`>vASy`=NfYpq^{mnSA%u%l_O<*2-S?T=@F!!n1AEF zDZgAH^i%6z+hk!r>fq?_JXa5u~+N+~p-|NlRH98NeTDk3tPVe{U^ZovD-LC8U>vy}(AK~P@yyoNi826Enr^t0aK|+65fjAHvD6Jq8 zvlYGSwI-_M*6|{#%RXCX(Vau{B-TIgSVQ_m?As_vhvM>+pD!bcGl1877Z};xJd;GV zl~>*j#gF`oagTZ#aGtnytYA8)wU2_No7MwVs??cohcguhA&s6!(ceh$+%e3uP8Ql% z^+`$?biHvZGai%LA$}~|jQwfOUibN5AlE|U7@lk9lCjO1o`mBk@pfT@p4VpysvYni z*5^6=4d5&GRH`i@I<`Ec=krW7jsnx0HrzZPIrHS5bS$L){1F-Qt!RJ+;7RJ_=*n*7 zTZ2+k`n>uP{;NFeb@IIBeBt8}8l?}{FBUJpexp))M}Rr8?zSZY|i^PVEoXYY)~ z-LjA-?)+T-lvQQAu|62Vq}4h%{y}U0a*NX|ZKed>N(U*s_xO@U1_=3F8wT~F`4A66 ztdLKBQ`C)(1_Wts*91FX^?PcIUNL8&V_5-V_aBtoD0N&W1%+j8S7yVdBO)A zz=4GLVV2Pm49drOOk6qc_VG#6T;o&u0oO&$oUD253N)*~dDZoNmAaO_-ee-CQq6ffL6rmGWW~ z=v^-;UXG*btj&(wS6SR36FDnX8Y$g%ek^X>-#v5rQF9bo#na~I**nGNgI_+#-;w=# zaLldxEoJUTE?pl^Y#gm0ByX{GJl%MH zBM?!9TqYRY7itkXZrDKy+8&i3Sr)&aR`d2}Wym-Dy>zwNKQ%rFJn4<=APiBxYaDJ{ z(>q4EX1GjHbdc-B-`UepXK;`@+jbcJou}rC?59#v1**6&;>yIPChEGHJvrC7a96iU zZqHX8C%J4?e+h<%kzn7rATrx6K(lyK#=L8uI1r?$MB=u7^#lh7P@MQ)HZkYxM*PyC z>1FTq{ets#N9rhYnWn`*`C`u7{iE_V{9_D~+NYV4Vx~?f`*N(a1KSw~xkO1x_jRfp zfG&eNdV%|*^L$TWz!n-ofN~9GW(hIdUg1)<5bl_CKKR~Rb2{$JW$_n0YTEnKzs1Eb zi#6{-#dVjAUh%e46hXq{m*(E77Ybq-r@x9nN=~>$kno63z{xa-o$^$(Lk7;wmek{F%1Ujl!=eWsSEu@?S} zO5X7T)8rM-yz)2)rq;|SF)@ZhRyj|)j43cYBo`=^qEQ#PJ6UFg5^nSjlJqNyt74c=gAPDDgq6-cCMCLVYer!ggt zEm`Kz&10Df)1!*qki1pQ*EK%d;3~=d%Tvcf?AIU9dGCv3^8ER>So;s{l?@yl_|F!J z%4QBE#5ODhLE?vfc`@yey;+P!&owDDb12EsT!lWHr!0Gn`{WCTrOL0CtqI|??@pdk zS#zYgIb{$O!GLpbZ(CSrnVG0~Z4o1LM0t~*b#k1loQd(_vqist`i##RjM1PFp&xW^ zUvlf$pL{)@L2*~#S1a?C_rORmzbl;O!jfIQAIuCxphaH>_HYonZ(aT@HyEY$X}l9; z=LothytQwCOSc{v6iMHm$tjb!vXH*(-RGNMQ5HJzE^=Aug*-CM;IaZ~i zd2(sGcdx=|wTOIdiRop0#_v#*2#n0%aX<1E`RTCHH_E>T>HD9|Ggeb_Ue?-3jqL_BTsgk;EGh5(?Tw1fG@tv{$DSC`t!;z8%0vrcu;#=2?EP)WdyXX4Mi;w(`Cw`t~ zX6A=3n&y8_MC<$;79v)K0k*r1zT|WRQ`YOEav)+>jFswNy({zk9|)h(?%LHmktzB) zvJn$^!AtAgTRoQ<&0!PIC7{pH*TqMO zZf{`(EB+K5UcEQghD{rPQKUWw zwUHuFa0=G}@kJAa8nr!`2=otaA6w8OX@vZ`AP@t{+eCu?{a?NCbT1X#{LR1vWw6XoBdu`4aIj(ZEIaV_N`A6TvZ+(c>@%pSFgk63Khd`^Xt+ zjq$|~Spw4D|BH%)#?fN(?|bS#&-Gu8U%XS?5s?&!A2~*>kvV4acxGm6*O~i4@Mkvt zvR$$J6iLuw%IDd=QR9#Vy;=(bbaz7}XCK?(kF7rcPK1@~N)JVnHF5DqJ}>eEH8Bi{ zK;1249adJE`mBui+R5fEa;}ShBBvw@Z%=c*`P=Yf6w`xtri$=&+Bl!yRmH6@Jv4tI z(l;Da+s*b-uNrNiax4)V6l#Y}^S~W)u6CXcX@3lDL-R@y6^$W`4O=FF!SlS8xt6~* zvUh%fZGGGwR1G8H#A^&Ym*yuzV20&|gMVDmKB*I%nVp?4aB!FGm?IK=E}9of1ZD2r z$+u|OUw;{Y+hCI*aIQU6t!}(NkNSNV05K4RdufO1MJu7P;a>xtHdH=BYz$~u6 zBjTjI)xK(CyF*JeWet-Hn@@fI1rg`H&K4n$jPz7VXIhSe zg~fX8d|O6GnnAC|F~NK7GUXXKrA?#sJ?JzOo|DBd36xzLU&gP7Y@7fssTw;L<*|&9 zYJA7(2y=oz=d#t~^3=WmMzQBPDrs#I*0Jhmujs!r%Ch%@{kD#Tr;Bgio?lt(+~B1) zrhiVxIs+Nq?+1pe#1**@%N;?$$HD8rc_qX&=$Xwwk7=kv9|5|fjcSN&I~=UfT}NrB})v-l$x0u1AATzou_iHQl% zYL{m6-PdzH{MY`&U)_CVeRVp$s0nopJ_7x4g9}s(a7Q$%g9JS|L0VyP?@rHH(n9F_VFGgC`lx%y&fwgSi$M9 zwSkI4oqQ;hgf7SoWXyjg_{*4^o5L6cCXVeudfZfBdn@H%zi46uvDQmb*8LkKk{K>d z*)SOzuC&X)eZ6I2)}27(0zw%m^Z*Gj8i22W*eVCwV?}rvn5}l@w%QLpAwT1KdB^zv zd0QI-I2Nwcuj-<1v2j?6Hlv%tY>f>pXOvJW@Hg}9>~DFr*XVb%yOrpNWO*_M=)+rza?y20BjW{w9H`=Xl)Q5 zk|;7~Qu~JBDG%q(0#MV|OlY$LJVs(N&@G_-Y2dW*w(znM-T9tGvL^QzZjS}Rfz=20 zf2sW{z&k0A1QO`;0I{Mv@a)LC|2BPn=kP}L1HvfO&IaCd7Wwha?WHqviR^wg!HTzk z9;lE>_*Hlg`$Pr&S(Vv1IP%QS5JaGOKYb;rgm^XU*r{t$*v4r)=vCUP^l(sNEY7wF zMY9sjeKs7=e?DAh>fS4d0}>$yfRf4_QN7fnO+fzVL8a2e5+(%;=0M`gcLaIg`9QaG z{WOb=9Ee*1FWo}z1_0HY!#VhHwCfPEq<)bu>X5;b`v~K7&og8!#t7TgEUImFI=q-V zq9lr?+esKCdn-<^oj*hN<|ZLkV?-#h1k9UTSmauKaDC+_2^Kj4b=QoM2$@{A-fC~G_1x%{W<(`@;5 z0G#16%-%;%u5XSS&Q>hACchX^^(xHEKP9qVTdrZIr$*`wJYE9Tx7FK z38(w{>C!;$Ar$g?kqWo;%?rl|0}EsPuUNJ_dZKCOFm)FZzb&mtiHTgg0(2bcUeG^f zkA2%NrJZmLXw+g8or!6t4}y{xNsx24^rv~60xWJgn!GhJ3~pzxY{S%CG`8puon#_2 z9(z+ReYeJAH5MMVJd3P#O2C>9RuQj+ASzKVT_i;(F9DKvmT0L}f~xVYPIQ}utaxgKj1ksrT}Ih95Ql_XAaDr z`eIm#30s08v9rC6wtvn`Iy;Dzd!7SMrj8GSUthvfIda)x&Mb)Q}DnJfYzfG{BSifzdMRG!Y zbh&aKqpZP#m|K2h5`V1P=v2u#Yk==qqTO*4I6F7Q>sBC1N6Pm$a^4zVWPkaG0E^@z7!K#r+|^R2a=l2z^WN&v9JMNS2U zVV>}E_j{87UNAa%Ft&j3zRd00dDGGa)GwK9cOqShlFrfvcP*0Tq6l?&*+#I@TC2vl z5!MDkqbI3cOqa6hA+Et7O425(Z>SMvUAbZ@TGI12C>?q>i`9Ydh+ASZ1p+nSECC`{ zeWy_z3%p9t8!mw%$B}_MJC7sU!h|5@TvGVNCMiTxY9k(&Ve>-`Bk)A@u#dL4J_MO2JJj7lEHPTv{@H*3UT{nI{zzR_17Pk_M45C(02si6_;3d? z0#NrLoo8rt|2%}yMBKZH#k0vaJT-yH7FV2c3jjU7OCYm+%g9L(9UP_M=~FM0T^#3X zV0O|uf9!PPt47I1zweNwHIA|xhZ#T|Q+{DbL8D(jHkHSe{IX0mg2n1n%t;?%9K+KU zBoU_Z>6WeH;TsHMmNd+a?9=_svxTSD^ljJ&j7)qd{K$Qn**5ArC_d*{>cHdlh;ljn zgLA;crE7^Zm;(~NQ{UZ#uOV0Me!pC2|0i4gKS$L3pWN@?r1t*^X_S5p)+@InXE5fA z0#NuOcQ8DgT5o8rz;Un0CDZ}hPy{--#B7OmUmh9;y!?gXUkOxk(3<+87V7XbLrzr7 z9gsyS9zYWDn}8}mEwITc+Nl2)2_6A>*Y$IU4a^4 z;z)iC%&~Ep(lQJo1VqNgWoOqmgJ6jy4#-6y^$N;Yv!I^f)m{$3=f9kIwD^rI{YzvP@S!0gAsK%~2p$84xQh{l867G(d5p)9(x2(cQX{UT=uSoxQhNZi<^Ce*7Pm{Lu2Voe zVtsA$mL*HTCoe*#aSn#=w;}CC&WCwHxWcN81WP+f>O6VSxJ^EuRMEO^5Dv8^C?NGbk>c>T)Q0zWI zLNh)-FUthn4GDGuTjFZDx4sXFzq9Be1J(2pQ0V+c|azX%!|cL=^P zUgu%BjlN3|?afk|m=qwE}cLEfa}tvB9g-!p0555pPHJQ0a6c`aY6H{&)vk& z;|v)a8(U60iSgJw=QGf#0pU(fJgr|btYh*p=}uZ=ZbJ1l_H!mvY8&k++<(f1=iD~g zujgWhM4JOdC9TD-loXrQjMRFTyVSp}df3BZ`6U+^gDn^K4fcJ{r#QaCpeZS}z7Z5V z|5{S>;DDS5KGsh+SFuJ4aUjn&FkNHu;6a`z2UUsYO~`ojndU)SZ=9jaAHed%SsfeY zWo71om?HQjo)fZ;jt`!W^yE8?9TFx89`xftZ?IR8&_nO20c8;{rm)h%ut@;hF5(22 z#Gvf6!)w#tXq$2X-drIl-TAE@{PBKc_F|XfyB`H0aW%pRD?yZ@?=S%<($KJT)O@3& zX(INC{E|&3L(8RoiZd|MF1=-!J@+QajM7hFU-H;CL+uoJ9H0 zO$+K0iFkR&H(Ms-q$2|m?9^um)Xyqdbc}#wvRrt16}S`MEzb0tclZ{3@yQ)}LnKS(kkq zu3NIHK0BtunJ%Ln3aS$q