From b8991903730ee01027df1de15fdaa27be53bd125 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sat, 23 Feb 2019 20:53:26 -0800 Subject: [PATCH 01/13] orc extension reworked to use apache orc map-reduce lib, moved to core extensions, support for flattenSpec, tests, docs --- .idea/misc.xml | 1 + distribution/pom.xml | 4 +- .../development/extensions-contrib/orc.md | 113 ----- .../development/extensions-core/orc.md | 147 +++++++ docs/content/development/extensions.md | 2 +- extensions-contrib/orc-extensions/pom.xml | 318 -------------- .../input/orc/OrcHadoopInputRowParser.java | 317 -------------- .../input/orc/DruidOrcInputFormatTest.java | 242 ----------- .../orc/OrcHadoopInputRowParserTest.java | 188 -------- .../input/orc/OrcIndexGeneratorJobTest.java | 404 ------------------ .../example/TestOrcFile.testDate1900.orc | Bin 0 -> 30941 bytes .../example/TestOrcFile.testDate2038.orc | Bin 0 -> 95787 bytes .../orc-file-11-format-hadoop-job.json | 85 ++++ .../example/orc-file-11-format.orc | Bin 0 -> 373336 bytes .../orc-extensions/example/orc_split_elim.orc | Bin 0 -> 246402 bytes .../example/orc_split_elim_hadoop_job.json | 64 +++ .../example/testDate1900_hadoop_job.json | 64 +++ .../example/testDate2038_hadoop_job.json | 64 +++ .../orc-extensions/example/test_1.orc | Bin 0 -> 624 bytes .../example/test_1_hadoop_job.json | 18 +- .../orc-extensions/example/test_2.orc | Bin 0 -> 983 bytes .../example/test_2_hadoop_job.json | 70 +++ extensions-core/orc-extensions/pom.xml | 72 ++++ .../data/input/orc/OrcExtensionsModule.java | 3 +- .../input/orc/OrcHadoopInputRowParser.java | 76 ++++ .../druid/data/input/orc/OrcParseSpec.java | 67 +++ .../data/input/orc/OrcStructConverter.java | 198 +++++++++ .../input/orc/OrcStructFlattenerMaker.java | 98 +++++ .../data/input/orc/OrcStructJsonProvider.java | 189 ++++++++ ...rg.apache.druid.initialization.DruidModule | 0 .../orc/OrcHadoopInputRowParserTest.java | 263 ++++++++++++ .../src/test/resources/log4j2.xml | 0 pom.xml | 2 +- 33 files changed, 1473 insertions(+), 1596 deletions(-) delete mode 100644 docs/content/development/extensions-contrib/orc.md create mode 100644 docs/content/development/extensions-core/orc.md delete mode 100644 extensions-contrib/orc-extensions/pom.xml delete mode 100644 extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java delete mode 100644 extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java delete mode 100644 extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java delete mode 100644 extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java create mode 100644 extensions-core/orc-extensions/example/TestOrcFile.testDate1900.orc create mode 100644 extensions-core/orc-extensions/example/TestOrcFile.testDate2038.orc create mode 100644 extensions-core/orc-extensions/example/orc-file-11-format-hadoop-job.json create mode 100644 extensions-core/orc-extensions/example/orc-file-11-format.orc create mode 100644 extensions-core/orc-extensions/example/orc_split_elim.orc create mode 100644 extensions-core/orc-extensions/example/orc_split_elim_hadoop_job.json create mode 100644 extensions-core/orc-extensions/example/testDate1900_hadoop_job.json create mode 100644 extensions-core/orc-extensions/example/testDate2038_hadoop_job.json create mode 100644 extensions-core/orc-extensions/example/test_1.orc rename extensions-contrib/orc-extensions/example/hadoop_orc_job.json => extensions-core/orc-extensions/example/test_1_hadoop_job.json (81%) create mode 100644 extensions-core/orc-extensions/example/test_2.orc create mode 100644 extensions-core/orc-extensions/example/test_2_hadoop_job.json create mode 100644 extensions-core/orc-extensions/pom.xml rename {extensions-contrib => extensions-core}/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java (96%) create mode 100644 extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java create mode 100644 extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcParseSpec.java create mode 100644 extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java create mode 100644 extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java create mode 100644 extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java rename {extensions-contrib => extensions-core}/orc-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule (100%) create mode 100644 extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java rename {extensions-contrib => extensions-core}/orc-extensions/src/test/resources/log4j2.xml (100%) diff --git a/.idea/misc.xml b/.idea/misc.xml index 9d0b0220e000..ada5175311cb 100644 --- a/.idea/misc.xml +++ b/.idea/misc.xml @@ -35,6 +35,7 @@ diff --git a/distribution/pom.xml b/distribution/pom.xml index 89df39f69b3b..435bfbdeac0d 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -149,6 +149,8 @@ -c org.apache.druid.extensions:mysql-metadata-storage -c + org.apache.druid.extensions:druid-orc-extensions + -c org.apache.druid.extensions:druid-parquet-extensions -c org.apache.druid.extensions:postgresql-metadata-storage @@ -295,8 +297,6 @@ -c org.apache.druid.extensions.contrib:druid-opentsdb-emitter -c - org.apache.druid.extensions.contrib:druid-orc-extensions - -c org.apache.druid.extensions.contrib:druid-rabbitmq -c org.apache.druid.extensions.contrib:druid-redis-cache diff --git a/docs/content/development/extensions-contrib/orc.md b/docs/content/development/extensions-contrib/orc.md deleted file mode 100644 index b5a56bb57fd6..000000000000 --- a/docs/content/development/extensions-contrib/orc.md +++ /dev/null @@ -1,113 +0,0 @@ ---- -layout: doc_page -title: "ORC" ---- - - - -# ORC - -To use this extension, make sure to [include](../../operations/including-extensions.html) `druid-orc-extensions`. - -This extension enables Druid to ingest and understand the Apache ORC data format offline. - -## ORC Hadoop Parser - -This is for batch ingestion using the HadoopDruidIndexer. The inputFormat of inputSpec in ioConfig must be set to `"org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat"`. - -|Field | Type | Description | Required| -|----------|-------------|----------------------------------------------------------------------------------------|---------| -|type | String | This should say `orc` | yes| -|parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Any parse spec that extends ParseSpec is possible but only their TimestampSpec and DimensionsSpec are used. | yes| -|typeString| String | String representation of ORC struct type info. If not specified, auto constructed from parseSpec but all metric columns are dropped | no| -|mapFieldNameFormat| String | String format for resolving the flatten map fields. Default is `_`. | no | - -For example of `typeString`, string column col1 and array of string column col2 is represented by `"struct>"`. - -Currently, it only supports java primitive types, array of java primitive types and map of java primitive types. Thus, compound types 'list' and 'map' in [ORC types](https://orc.apache.org/docs/types.html) are supported. Note that, list of list is not supported, nor map of compound types. For map types, values will be exploded to several columns where column names will be resolved via `mapFieldNameFormat`. - -For example of hadoop indexing: - -```json -{ - "type": "index_hadoop", - "spec": { - "ioConfig": { - "type": "hadoop", - "inputSpec": { - "type": "static", - "inputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat", - "paths": "/data/path/in/HDFS/" - }, - "metadataUpdateSpec": { - "type": "postgresql", - "connectURI": "jdbc:postgresql://localhost/druid", - "user" : "druid", - "password" : "asdf", - "segmentTable": "druid_segments" - }, - "segmentOutputPath": "tmp/segments" - }, - "dataSchema": { - "dataSource": "no_metrics", - "parser": { - "type": "orc", - "parseSpec": { - "format": "timeAndDims", - "timestampSpec": { - "column": "time", - "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - "name" - ], - "dimensionExclusions": [], - "spatialDimensions": [] - } - }, - "typeString": "struct", - "mapFieldNameFormat": "_" - }, - "metricsSpec": [{ - "type": "count", - "name": "count" - }], - "granularitySpec": { - "type": "uniform", - "segmentGranularity": "DAY", - "queryGranularity": "ALL", - "intervals": ["2015-12-31/2016-01-02"] - } - }, - "tuningConfig": { - "type": "hadoop", - "workingPath": "tmp/working_path", - "partitionsSpec": { - "targetPartitionSize": 5000000 - }, - "jobProperties" : {}, - "leaveIntermediate": true - } - } -} -``` - -Almost all the fields listed above are required, including `inputFormat`, `metadataUpdateSpec`(`type`, `connectURI`, `user`, `password`, `segmentTable`). Set `jobProperties` to make hdfs path timezone unrelated. diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md new file mode 100644 index 000000000000..9e75c0ac9bc9 --- /dev/null +++ b/docs/content/development/extensions-core/orc.md @@ -0,0 +1,147 @@ +--- +layout: doc_page +title: "Druid ORC Extension" +--- + + + +# Druid ORC Extension + +This module extends [Druid Hadoop based indexing](../../ingestion/hadoop.html) to ingest data directly from offline +Apache ORC files. + +To use this extension, make sure to [include](../../operations/including-extensions.html) `druid-orc-extensions`. + +## ORC Hadoop Parser + +The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"org.apache.orc.mapreduce.OrcInputFormat"`. + + +|Field | Type | Description | Required| +|----------|-------------|----------------------------------------------------------------------------------------|---------| +|type | String | This should say `orc` | yes| +|parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Any parse spec that extends ParseSpec is possible but only their TimestampSpec and DimensionsSpec are used. | yes| + +The parser support auto field discovery and flattening if provided with a +[flattenSpec](../../ingestion/flatten-json.html) with `orc` as the `format`. +[All column types](https://orc.apache.org/docs/types.html) with the exception of `union` types are supported. Columns of + `list` type if filled with primitives may be used as a multi-value dimension, or specific elements can be extracted with +`flattenSpec` expressions. Likewise, primitive fields may be extracted from `map` and `struct` types in the same manner. + +### Examples + +#### `orc` parser, `orc` parseSpec + +```json +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "path/to/file.orc" + }, + ... + }, + "dataSchema": { + "dataSource": "example", + "parser": { + "type": "orc", + "parseSpec": { + "format": "orc", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "nestedDim", + "expr": "$.nestedData.dim1" + }, + { + "type": "path", + "name": "listDimFirstItem", + "expr": "$.listDim[1]" + } + ] + }, + "timestampSpec": { + "column": "timestamp", + "format": "millis" + }, + "dimensionsSpec": { + "dimensions": [], + "dimensionExclusions": [], + "spatialDimensions": [] + } + } + }, + ... + }, + "tuningConfig": + } + } +} +``` + +#### `orc` parser, `timeAndDims` parseSpec +```json +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "path/to/file.orc" + }, + ... + }, + "dataSchema": { + "dataSource": "example", + "parser": { + "type": "orc", + "parseSpec": { + "format": "timeAndDims", + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "dim1", + "dim2", + "dim3", + "listDim" + ], + "dimensionExclusions": [], + "spatialDimensions": [] + } + } + }, + ... + }, + "tuningConfig": + } +} + +``` diff --git a/docs/content/development/extensions.md b/docs/content/development/extensions.md index a6d3a7b3913d..9d3d72bfd7f6 100644 --- a/docs/content/development/extensions.md +++ b/docs/content/development/extensions.md @@ -55,6 +55,7 @@ Core extensions are maintained by Druid committers. |druid-kerberos|Kerberos authentication for druid nodes.|[link](../development/extensions-core/druid-kerberos.html)| |druid-lookups-cached-global|A module for [lookups](../querying/lookups.html) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.html)| |druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.html)| +|druid-orc-extensions|Support for data in Apache Orc data format.|[link](../development/extensions-core/orc.html)| |druid-parquet-extensions|Support for data in Apache Parquet data format. Requires druid-avro-extensions to be loaded.|[link](../development/extensions-core/parquet.html)| |druid-protobuf-extensions| Support for data in Protobuf data format.|[link](../development/extensions-core/protobuf.html)| |druid-s3-extensions|Interfacing with data in AWS S3, and using S3 as deep storage.|[link](../development/extensions-core/s3.html)| @@ -82,7 +83,6 @@ All of these community extensions can be downloaded using *pull-deps* with the c |druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.html)| |druid-distinctcount|DistinctCount aggregator|[link](../development/extensions-contrib/distinctcount.html)| |druid-kafka-eight-simpleConsumer|Kafka ingest firehose (low level consumer)(deprecated).|[link](../development/extensions-contrib/kafka-simple.html)| -|druid-orc-extensions|Support for data in Apache Orc data format.|[link](../development/extensions-contrib/orc.html)| |druid-rabbitmq|RabbitMQ firehose.|[link](../development/extensions-contrib/rabbitmq.html)| |druid-redis-cache|A cache implementation for Druid based on Redis.|[link](../development/extensions-contrib/redis-cache.html)| |druid-rocketmq|RocketMQ firehose.|[link](../development/extensions-contrib/rocketmq.html)| diff --git a/extensions-contrib/orc-extensions/pom.xml b/extensions-contrib/orc-extensions/pom.xml deleted file mode 100644 index 944f061de7fb..000000000000 --- a/extensions-contrib/orc-extensions/pom.xml +++ /dev/null @@ -1,318 +0,0 @@ - - - - org.apache.druid.extensions.contrib - druid-orc-extensions - druid-orc-extensions - druid-orc-extensions - - - druid - org.apache.druid - 0.15.0-incubating-SNAPSHOT - ../../pom.xml - - 4.0.0 - - - - org.apache.druid - druid-indexing-hadoop - ${project.parent.version} - provided - - - org.apache.hadoop - hadoop-client - ${hadoop.compile.version} - provided - - - org.apache.hive - hive-exec - ${hive.version} - - - commons-httpclient - commons-httpclient - - - commons-logging - commons-logging - - - org.apache.hive - hive-ant - - - org.apache.hive - hive-common - - - org.apache.hive - hive-vector-code-gen - - - org.apache.hive - hive-metastore - - - org.apache.hive - hive-service-rpc - - - org.apache.hive - hive-llap-client - - - org.apache.hive - hive-llap-tez - - - org.apache.hive - hive-shims - - - org.apache.hive - hive-spark-client - - - com.esotericsoftware - kryo-shaded - - - com.google.protobuf - protobuf-java - - - org.apache.parquet - parquet-hadoop-bundle - - - commons-codec - commons-codec - - - commons-io - commons-io - - - org.apache.commons - commons-lang3 - - - commons-lang - commons-lang - - - javolution - javolution - - - org.apache.logging.log4j - log4j-1.2-api - - - org.apache.logging.log4j - log4j-slf4j-impl - - - org.antlr - antlr-runtime - - - org.antlr - ST4 - - - org.apache.avro - avro - - - org.apache.avro - avro-mapred - - - org.apache.ant - ant - - - org.apache.commons - commons-compress - - - org.apache.thrift - libfb303 - - - org.apache.hadoop - hadoop-common - - - org.apache.hadoop - hadoop-archives - - - org.apache.hadoop - hadoop-yarn-registry - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - org.apache.hadoop - hadoop-mapreduce-client-common - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hadoop - hadoop-yarn-api - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-yarn-client - - - org.apache.orc - orc-tools - - - org.apache.ivy - ivy - - - org.apache.thrift - libthrift - - - org.apache.zookeeper - zookeeper - - - org.apache.curator - curator-framework - - - org.apache.curator - apache-curator - - - org.codehaus.groovy - groovy-all - - - org.jodd - jodd-core - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - org.datanucleus - datanucleus-core - - - org.apache.calcite - calcite-core - - - org.apache.calcite - calcite-druid - - - org.apache.calcite - calcite-avatica - - - org.apache.calcite.avatica - avatica - - - com.google.guava - guava - - - com.googlecode.javaewah - JavaEWAH - - - com.google.code.gson - gson - - - com.tdunning - json - - - stax - stax-api - - - net.sf.opencsv - opencsv - - - org.apache.hive - hive-standalone-metastore-server - - - org.slf4j - slf4j-api - - - oro - oro - - - org.apache.velocity - velocity - - - jline - jline - - - - - junit - junit - test - - - diff --git a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java deleted file mode 100644 index 6d3340ca81a8..000000000000 --- a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ /dev/null @@ -1,317 +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. - */ - -package org.apache.druid.data.input.orc; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.ql.io.orc.OrcSerde; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.hive.serde2.SerDeException; -import org.apache.hadoop.hive.serde2.io.DateWritable; -import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; -import org.apache.hadoop.hive.serde2.objectinspector.StructField; -import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.joda.time.DateTime; - -import javax.annotation.Nullable; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.stream.Collectors; - -public class OrcHadoopInputRowParser implements InputRowParser -{ - - static final String MAP_CHILD_TAG = ""; - static final String MAP_PARENT_TAG = ""; - static final String DEFAULT_MAP_FIELD_NAME_FORMAT = MAP_PARENT_TAG + "_" + MAP_CHILD_TAG; - - - private final ParseSpec parseSpec; - private final String typeString; - private final String mapFieldNameFormat; - private final String mapParentFieldNameFormat; - private final List dimensions; - private final StructObjectInspector oip; - - - - @JsonCreator - public OrcHadoopInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec, - @JsonProperty("typeString") String typeString, - @JsonProperty("mapFieldNameFormat") String mapFieldNameFormat - ) - { - this.parseSpec = parseSpec; - this.typeString = typeString == null ? typeStringFromParseSpec(parseSpec) : typeString; - this.mapFieldNameFormat = - mapFieldNameFormat == null || - !mapFieldNameFormat.contains(MAP_PARENT_TAG) || - !mapFieldNameFormat.contains(MAP_CHILD_TAG) ? DEFAULT_MAP_FIELD_NAME_FORMAT : mapFieldNameFormat; - this.mapParentFieldNameFormat = StringUtils.replace(this.mapFieldNameFormat, MAP_PARENT_TAG, "%s"); - this.dimensions = parseSpec.getDimensionsSpec().getDimensionNames(); - this.oip = makeObjectInspector(this.typeString); - } - - @SuppressWarnings("ArgumentParameterSwap") - @Override - public List parseBatch(OrcStruct input) - { - Map map = new HashMap<>(); - List fields = oip.getAllStructFieldRefs(); - for (StructField field : fields) { - ObjectInspector objectInspector = field.getFieldObjectInspector(); - switch (objectInspector.getCategory()) { - case PRIMITIVE: - PrimitiveObjectInspector primitiveObjectInspector = (PrimitiveObjectInspector) objectInspector; - map.put( - field.getFieldName(), - coercePrimitiveObject( - primitiveObjectInspector, - oip.getStructFieldData(input, field) - ) - ); - break; - case LIST: // array case - only 1-depth array supported yet - ListObjectInspector listObjectInspector = (ListObjectInspector) objectInspector; - map.put( - field.getFieldName(), - getListObject(listObjectInspector, oip.getStructFieldData(input, field)) - ); - break; - case MAP: - MapObjectInspector mapObjectInspector = (MapObjectInspector) objectInspector; - getMapObject(field.getFieldName(), mapObjectInspector, oip.getStructFieldData(input, field), map); - break; - default: - break; - } - } - - TimestampSpec timestampSpec = parseSpec.getTimestampSpec(); - DateTime dateTime = timestampSpec.extractTimestamp(map); - - final List dimensions; - if (!this.dimensions.isEmpty()) { - dimensions = this.dimensions; - } else { - dimensions = Lists.newArrayList( - Sets.difference(map.keySet(), parseSpec.getDimensionsSpec().getDimensionExclusions()) - ); - } - return ImmutableList.of(new MapBasedInputRow(dateTime, dimensions, map)); - } - - private List getListObject(ListObjectInspector listObjectInspector, Object listObject) - { - if (listObjectInspector.getListLength(listObject) < 0) { - return null; - } - List objectList = listObjectInspector.getList(listObject); - List list = null; - ObjectInspector child = listObjectInspector.getListElementObjectInspector(); - switch (child.getCategory()) { - case PRIMITIVE: - final PrimitiveObjectInspector primitiveObjectInspector = (PrimitiveObjectInspector) child; - list = objectList.stream() - .map(input -> coercePrimitiveObject(primitiveObjectInspector, input)) - .collect(Collectors.toList()); - break; - default: - break; - } - - return list; - } - - private void getMapObject(String parentName, MapObjectInspector mapObjectInspector, Object mapObject, Map parsedMap) - { - if (mapObjectInspector.getMapSize(mapObject) < 0) { - return; - } - String mapChildFieldNameFormat = StringUtils.replace( - StringUtils.format(mapParentFieldNameFormat, parentName), - MAP_CHILD_TAG, - "%s" - ); - - Map objectMap = mapObjectInspector.getMap(mapObject); - PrimitiveObjectInspector key = (PrimitiveObjectInspector) mapObjectInspector.getMapKeyObjectInspector(); - PrimitiveObjectInspector value = (PrimitiveObjectInspector) mapObjectInspector.getMapValueObjectInspector(); - - objectMap.forEach((k, v) -> { - String resolvedFieldName = StringUtils.format(mapChildFieldNameFormat, key.getPrimitiveJavaObject(k).toString()); - parsedMap.put(resolvedFieldName, value.getPrimitiveJavaObject(v)); - }); - } - - @JsonProperty - public String getMapFieldNameFormat() - { - return mapFieldNameFormat; - } - - @Override - @JsonProperty - public ParseSpec getParseSpec() - { - return parseSpec; - } - - @JsonProperty - public String getTypeString() - { - return typeString; - } - - @Override - public InputRowParser withParseSpec(ParseSpec parseSpec) - { - return new OrcHadoopInputRowParser(parseSpec, typeString, null); - } - - @Override - public boolean equals(final Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final OrcHadoopInputRowParser that = (OrcHadoopInputRowParser) o; - return Objects.equals(parseSpec, that.parseSpec) && - Objects.equals(typeString, that.typeString); - } - - @Override - public int hashCode() - { - return Objects.hash(parseSpec, typeString); - } - - @Override - public String toString() - { - return "OrcHadoopInputRowParser{" + - "parseSpec=" + parseSpec + - ", typeString='" + typeString + '\'' + - '}'; - } - - @VisibleForTesting - static String typeStringFromParseSpec(ParseSpec parseSpec) - { - StringBuilder builder = new StringBuilder("struct<"); - builder.append(parseSpec.getTimestampSpec().getTimestampColumn()).append(":string"); - // the typeString seems positionally dependent, so repeated timestamp column causes incorrect mapping - if (parseSpec.getDimensionsSpec().getDimensionNames().size() > 0) { - builder.append(","); - builder.append(String.join( - ":string,", - parseSpec.getDimensionsSpec() - .getDimensionNames() - .stream() - .filter(s -> !s.equals(parseSpec.getTimestampSpec().getTimestampColumn())) - .collect(Collectors.toList()))); - builder.append(":string"); - } - builder.append(">"); - - return builder.toString(); - } - - private static Object coercePrimitiveObject(final PrimitiveObjectInspector inspector, final Object object) - { - if (object instanceof HiveDecimalWritable) { - // inspector on HiveDecimal rounds off to integer for some reason. - return ((HiveDecimalWritable) object).getHiveDecimal().doubleValue(); - } else if (object instanceof DateWritable) { - return object.toString(); - } else { - return inspector.getPrimitiveJavaObject(object); - } - } - - private static StructObjectInspector makeObjectInspector(final String typeString) - { - final OrcSerde serde = new OrcSerde(); - - TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeString); - Preconditions.checkArgument( - typeInfo instanceof StructTypeInfo, - StringUtils.format("typeString should be struct type but not [%s]", typeString) - ); - Properties table = getTablePropertiesFromStructTypeInfo((StructTypeInfo) typeInfo); - serde.initialize(new Configuration(), table); - try { - return (StructObjectInspector) serde.getObjectInspector(); - } - catch (SerDeException e) { - throw new RuntimeException(e); - } - } - - private static Properties getTablePropertiesFromStructTypeInfo(StructTypeInfo structTypeInfo) - { - Properties table = new Properties(); - table.setProperty("columns", String.join(",", structTypeInfo.getAllStructFieldNames())); - table.setProperty("columns.types", String.join( - ",", - Lists.transform( - structTypeInfo.getAllStructFieldTypeInfos(), - new Function() - { - @Nullable - @Override - public String apply(@Nullable TypeInfo typeInfo) - { - return typeInfo.getTypeName(); - } - } - ) - )); - - return table; - } -} diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java deleted file mode 100644 index 57462d2ef204..000000000000 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/DruidOrcInputFormatTest.java +++ /dev/null @@ -1,242 +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. - */ - -package org.apache.druid.data.input.orc; - -import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.indexer.HadoopDruidIndexerConfig; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.lib.input.FileSplit; -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import org.apache.hadoop.util.ReflectionUtils; -import org.apache.orc.CompressionKind; -import org.apache.orc.OrcFile; -import org.apache.orc.TypeDescription; -import org.apache.orc.Writer; -import org.joda.time.DateTime; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.concurrent.TimeUnit; - -public class DruidOrcInputFormatTest -{ - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - String timestamp = "2016-01-01T00:00:00.000Z"; - String col1 = "bar"; - String[] col2 = {"dat1", "dat2", "dat3"}; - double val1 = 1.1; - Job job; - HadoopDruidIndexerConfig config; - File testFile; - Path path; - FileSplit split; - - @Before - public void setUp() throws IOException - { - Configuration conf = new Configuration(); - job = Job.getInstance(conf); - - config = HadoopDruidIndexerConfig.fromFile(new File( - "example/hadoop_orc_job.json")); - - config.intoConfiguration(job); - - testFile = makeOrcFile(); - path = new Path(testFile.getAbsoluteFile().toURI()); - split = new FileSplit(path, 0, testFile.length(), null); - - } - - @Test - public void testRead() throws IOException, InterruptedException - { - InputFormat inputFormat = ReflectionUtils.newInstance(OrcNewInputFormat.class, job.getConfiguration()); - - TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); - RecordReader reader = inputFormat.createRecordReader(split, context); - InputRowParser parser = (InputRowParser) config.getParser(); - - reader.initialize(split, context); - - reader.nextKeyValue(); - - OrcStruct data = (OrcStruct) reader.getCurrentValue(); - - MapBasedInputRow row = (MapBasedInputRow) parser.parseBatch(data).get(0); - - Assert.assertTrue(row.getEvent().keySet().size() == 4); - Assert.assertEquals(DateTimes.of(timestamp), row.getTimestamp()); - Assert.assertEquals(parser.getParseSpec().getDimensionsSpec().getDimensionNames(), row.getDimensions()); - Assert.assertEquals(col1, row.getEvent().get("col1")); - Assert.assertEquals(Arrays.asList(col2), row.getDimension("col2")); - - reader.close(); - } - - @Test - public void testReadDateColumn() throws IOException, InterruptedException - { - File testFile2 = makeOrcFileWithDate(); - Path path = new Path(testFile2.getAbsoluteFile().toURI()); - FileSplit split = new FileSplit(path, 0, testFile2.length(), null); - - InputFormat inputFormat = ReflectionUtils.newInstance(OrcNewInputFormat.class, job.getConfiguration()); - - TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); - RecordReader reader = inputFormat.createRecordReader(split, context); - InputRowParser parser = (InputRowParser) config.getParser(); - - reader.initialize(split, context); - - reader.nextKeyValue(); - - OrcStruct data = (OrcStruct) reader.getCurrentValue(); - - MapBasedInputRow row = (MapBasedInputRow) parser.parseBatch(data).get(0); - - Assert.assertTrue(row.getEvent().keySet().size() == 4); - Assert.assertEquals(DateTimes.of(timestamp), row.getTimestamp()); - Assert.assertEquals(parser.getParseSpec().getDimensionsSpec().getDimensionNames(), row.getDimensions()); - Assert.assertEquals(col1, row.getEvent().get("col1")); - Assert.assertEquals(Arrays.asList(col2), row.getDimension("col2")); - - reader.close(); - } - - private File makeOrcFile() throws IOException - { - final File dir = temporaryFolder.newFolder(); - final File testOrc = new File(dir, "test.orc"); - TypeDescription schema = TypeDescription.createStruct() - .addField("timestamp", TypeDescription.createString()) - .addField("col1", TypeDescription.createString()) - .addField("col2", TypeDescription.createList(TypeDescription.createString())) - .addField("val1", TypeDescription.createFloat()); - Configuration conf = new Configuration(); - Writer writer = OrcFile.createWriter( - new Path(testOrc.getPath()), - OrcFile.writerOptions(conf) - .setSchema(schema) - .stripeSize(100000) - .bufferSize(10000) - .compress(CompressionKind.ZLIB) - .version(OrcFile.Version.CURRENT) - ); - VectorizedRowBatch batch = schema.createRowBatch(); - batch.size = 1; - ((BytesColumnVector) batch.cols[0]).setRef( - 0, - StringUtils.toUtf8(timestamp), - 0, - timestamp.length() - ); - ((BytesColumnVector) batch.cols[1]).setRef(0, StringUtils.toUtf8(col1), 0, col1.length()); - - ListColumnVector listColumnVector = (ListColumnVector) batch.cols[2]; - listColumnVector.childCount = col2.length; - listColumnVector.lengths[0] = 3; - for (int idx = 0; idx < col2.length; idx++) { - ((BytesColumnVector) listColumnVector.child).setRef( - idx, - StringUtils.toUtf8(col2[idx]), - 0, - col2[idx].length() - ); - } - - ((DoubleColumnVector) batch.cols[3]).vector[0] = val1; - writer.addRowBatch(batch); - writer.close(); - - return testOrc; - } - - private File makeOrcFileWithDate() throws IOException - { - final File dir = temporaryFolder.newFolder(); - final File testOrc = new File(dir, "test-2.orc"); - TypeDescription schema = TypeDescription.createStruct() - .addField("timestamp", TypeDescription.createDate()) - .addField("col1", TypeDescription.createString()) - .addField("col2", TypeDescription.createList(TypeDescription.createString())) - .addField("val1", TypeDescription.createFloat()); - Configuration conf = new Configuration(); - Writer writer = OrcFile.createWriter( - new Path(testOrc.getPath()), - OrcFile.writerOptions(conf) - .setSchema(schema) - .stripeSize(100000) - .bufferSize(10000) - .compress(CompressionKind.ZLIB) - .version(OrcFile.Version.CURRENT) - ); - VectorizedRowBatch batch = schema.createRowBatch(); - batch.size = 1; - DateTime ts = DateTimes.of(timestamp); - - // date is stored as long column vector with number of days since epoch - ((LongColumnVector) batch.cols[0]).vector[0] = - TimeUnit.MILLISECONDS.toDays(ts.minus(DateTimes.EPOCH.getMillis()).getMillis()); - - ((BytesColumnVector) batch.cols[1]).setRef(0, StringUtils.toUtf8(col1), 0, col1.length()); - - ListColumnVector listColumnVector = (ListColumnVector) batch.cols[2]; - listColumnVector.childCount = col2.length; - listColumnVector.lengths[0] = 3; - for (int idx = 0; idx < col2.length; idx++) { - ((BytesColumnVector) listColumnVector.child).setRef( - idx, - StringUtils.toUtf8(col2[idx]), - 0, - col2[idx].length() - ); - } - - ((DoubleColumnVector) batch.cols[3]).vector[0] = val1; - writer.addRowBatch(batch); - writer.close(); - - return testOrc; - } -} diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java deleted file mode 100644 index 3b7f56928bb9..000000000000 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java +++ /dev/null @@ -1,188 +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. - */ - -package org.apache.druid.data.input.orc; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.inject.Binder; -import com.google.inject.Injector; -import com.google.inject.Module; -import com.google.inject.name.Names; -import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.ParseSpec; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.guice.GuiceInjectors; -import org.apache.druid.initialization.Initialization; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.hadoop.hive.common.type.HiveDecimal; -import org.apache.hadoop.hive.ql.io.orc.OrcStruct; -import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; -import org.apache.hadoop.hive.serde2.objectinspector.SettableStructObjectInspector; -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; -import org.apache.hadoop.io.FloatWritable; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; -import java.math.BigDecimal; - -public class OrcHadoopInputRowParserTest -{ - Injector injector; - ObjectMapper mapper = new DefaultObjectMapper(); - - @Before - public void setUp() - { - injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("test"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - } - }, - new OrcExtensionsModule() - ) - ); - mapper = injector.getInstance(ObjectMapper.class); - } - - @Test - public void testSerde() throws IOException - { - String parserString = "{\n" + - " \"type\": \"orc\",\n" + - " \"parseSpec\": {\n" + - " \"format\": \"timeAndDims\",\n" + - " \"timestampSpec\": {\n" + - " \"column\": \"timestamp\",\n" + - " \"format\": \"auto\"\n" + - " },\n" + - " \"dimensionsSpec\": {\n" + - " \"dimensions\": [\n" + - " \"col1\",\n" + - " \"col2\"\n" + - " ],\n" + - " \"dimensionExclusions\": [],\n" + - " \"spatialDimensions\": []\n" + - " }\n" + - " },\n" + - " \"typeString\": \"struct,val1:float>\"\n" + - " }"; - - InputRowParser parser = mapper.readValue(parserString, InputRowParser.class); - InputRowParser expected = new OrcHadoopInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), - new DimensionsSpec( - ImmutableList.of(new StringDimensionSchema("col1"), new StringDimensionSchema("col2")), - null, - null - ) - ), - "struct,val1:float>", - null - ); - - Assert.assertEquals(expected, parser); - } - - @Test - public void testTypeFromParseSpec() - { - ParseSpec parseSpec = new TimeAndDimsParseSpec( - new TimestampSpec( - "timestamp", - "auto", - null - ), - new DimensionsSpec( - ImmutableList.of(new StringDimensionSchema("col1"), new StringDimensionSchema("col2")), - null, - null - ) - ); - String typeString = OrcHadoopInputRowParser.typeStringFromParseSpec(parseSpec); - String expected = "struct"; - - Assert.assertEquals(expected, typeString); - } - - @Test - public void testParse() - { - final String typeString = "struct,col3:float,col4:bigint,col5:decimal,col6:array,col7:map>"; - final OrcHadoopInputRowParser parser = new OrcHadoopInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), - new DimensionsSpec(null, null, null) - ), - typeString, - "-" - ); - - final SettableStructObjectInspector oi = (SettableStructObjectInspector) OrcStruct.createObjectInspector( - TypeInfoUtils.getTypeInfoFromTypeString(typeString) - ); - final OrcStruct struct = (OrcStruct) oi.create(); - struct.setNumFields(8); - oi.setStructFieldData(struct, oi.getStructFieldRef("timestamp"), new Text("2000-01-01")); - oi.setStructFieldData(struct, oi.getStructFieldRef("col1"), new Text("foo")); - oi.setStructFieldData(struct, oi.getStructFieldRef("col2"), ImmutableList.of(new Text("foo"), new Text("bar"))); - oi.setStructFieldData(struct, oi.getStructFieldRef("col3"), new FloatWritable(1.5f)); - oi.setStructFieldData(struct, oi.getStructFieldRef("col4"), new LongWritable(2)); - oi.setStructFieldData( - struct, - oi.getStructFieldRef("col5"), - new HiveDecimalWritable(HiveDecimal.create(BigDecimal.valueOf(3.5d))) - ); - oi.setStructFieldData(struct, oi.getStructFieldRef("col6"), null); - oi.setStructFieldData(struct, oi.getStructFieldRef("col7"), ImmutableMap.of(new Text("subcol7"), new Text("subval7"))); - - final InputRow row = parser.parseBatch(struct).get(0); - Assert.assertEquals("timestamp", DateTimes.of("2000-01-01"), row.getTimestamp()); - Assert.assertEquals("col1", "foo", row.getRaw("col1")); - Assert.assertEquals("col2", ImmutableList.of("foo", "bar"), row.getRaw("col2")); - Assert.assertEquals("col3", 1.5f, row.getRaw("col3")); - Assert.assertEquals("col4", 2L, row.getRaw("col4")); - Assert.assertEquals("col5", 3.5d, row.getRaw("col5")); - Assert.assertNull("col6", row.getRaw("col6")); - Assert.assertEquals("col7-subcol7", "subval7", row.getRaw("col7-subcol7")); - } -} diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java deleted file mode 100644 index 6108fbad4906..000000000000 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ /dev/null @@ -1,404 +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. - */ - -package org.apache.druid.data.input.orc; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.jsontype.NamedType; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.io.Files; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.InputRowParser; -import org.apache.druid.data.input.impl.TimeAndDimsParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.HadoopDruidIndexerConfig; -import org.apache.druid.indexer.HadoopIOConfig; -import org.apache.druid.indexer.HadoopIngestionSpec; -import org.apache.druid.indexer.HadoopTuningConfig; -import org.apache.druid.indexer.HadoopyShardSpec; -import org.apache.druid.indexer.IndexGeneratorJob; -import org.apache.druid.indexer.JobHelper; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.QueryableIndexIndexableAdapter; -import org.apache.druid.segment.RowIterator; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector; -import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch; -import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.orc.CompressionKind; -import org.apache.orc.OrcFile; -import org.apache.orc.TypeDescription; -import org.apache.orc.Writer; -import org.joda.time.DateTime; -import org.joda.time.DateTimeComparator; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.BufferedOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.TreeMap; -import java.util.zip.ZipEntry; -import java.util.zip.ZipInputStream; - -public class OrcIndexGeneratorJobTest -{ - private static final AggregatorFactory[] aggs = { - new LongSumAggregatorFactory("visited_num", "visited_num"), - new HyperUniquesAggregatorFactory("unique_hosts", "host") - }; - - @Rule - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private ObjectMapper mapper; - private HadoopDruidIndexerConfig config; - private final String dataSourceName = "website"; - private final List data = ImmutableList.of( - "2014102200,a.example.com,100", - "2014102200,b.exmaple.com,50", - "2014102200,c.example.com,200", - "2014102200,d.example.com,250", - "2014102200,e.example.com,123", - "2014102200,f.example.com,567", - "2014102200,g.example.com,11", - "2014102200,h.example.com,251", - "2014102200,i.example.com,963", - "2014102200,j.example.com,333", - "2014102212,a.example.com,100", - "2014102212,b.exmaple.com,50", - "2014102212,c.example.com,200", - "2014102212,d.example.com,250", - "2014102212,e.example.com,123", - "2014102212,f.example.com,567", - "2014102212,g.example.com,11", - "2014102212,h.example.com,251", - "2014102212,i.example.com,963", - "2014102212,j.example.com,333" - ); - private final Interval interval = Intervals.of("2014-10-22T00:00:00Z/P1D"); - private File dataRoot; - private File outputRoot; - private Integer[][][] shardInfoForEachSegment = new Integer[][][]{ - { - {0, 4}, - {1, 4}, - {2, 4}, - {3, 4} - } - }; - private final InputRowParser inputRowParser = new OrcHadoopInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null) - ), - "struct", - null - ); - - private File writeDataToLocalOrcFile(File outputDir, List data) throws IOException - { - File outputFile = new File(outputDir, "test.orc"); - TypeDescription schema = TypeDescription.createStruct() - .addField("timestamp", TypeDescription.createString()) - .addField("host", TypeDescription.createString()) - .addField("visited_num", TypeDescription.createInt()); - Configuration conf = new Configuration(); - Writer writer = OrcFile.createWriter( - new Path(outputFile.getPath()), - OrcFile.writerOptions(conf) - .setSchema(schema) - .stripeSize(100000) - .bufferSize(10000) - .compress(CompressionKind.ZLIB) - .version(OrcFile.Version.CURRENT) - ); - VectorizedRowBatch batch = schema.createRowBatch(); - batch.size = data.size(); - for (int idx = 0; idx < data.size(); idx++) { - String line = data.get(idx); - String[] lineSplit = line.split(","); - ((BytesColumnVector) batch.cols[0]).setRef( - idx, - StringUtils.toUtf8(lineSplit[0]), - 0, - lineSplit[0].length() - ); - ((BytesColumnVector) batch.cols[1]).setRef( - idx, - StringUtils.toUtf8(lineSplit[1]), - 0, - lineSplit[1].length() - ); - ((LongColumnVector) batch.cols[2]).vector[idx] = Long.parseLong(lineSplit[2]); - } - writer.addRowBatch(batch); - writer.close(); - - return outputFile; - } - - @Before - public void setUp() throws Exception - { - mapper = HadoopDruidIndexerConfig.JSON_MAPPER; - mapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); - - dataRoot = temporaryFolder.newFolder("data"); - outputRoot = temporaryFolder.newFolder("output"); - File dataFile = writeDataToLocalOrcFile(dataRoot, data); - - HashMap inputSpec = new HashMap(); - inputSpec.put("paths", dataFile.getCanonicalPath()); - inputSpec.put("type", "static"); - inputSpec.put("inputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat"); - - config = new HadoopDruidIndexerConfig( - new HadoopIngestionSpec( - new DataSchema( - dataSourceName, - mapper.convertValue( - inputRowParser, - Map.class - ), - aggs, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), - null, - mapper - ), - new HadoopIOConfig( - ImmutableMap.copyOf(inputSpec), - null, - outputRoot.getCanonicalPath() - ), - new HadoopTuningConfig( - outputRoot.getCanonicalPath(), - null, - null, - null, - null, - null, - null, - true, - false, - false, - false, - ImmutableMap.of(MRJobConfig.NUM_REDUCES, "0"), //verifies that set num reducers is ignored - false, - true, - null, - true, - null, - false, - false, - null, - null, - null - ) - ) - ); - config.setShardSpecs( - loadShardSpecs(shardInfoForEachSegment) - ); - config = HadoopDruidIndexerConfig.fromSpec(config.getSchema()); - } - - @Test - public void testIndexGeneratorJob() throws IOException - { - verifyJob(new IndexGeneratorJob(config)); - } - - private void verifyJob(IndexGeneratorJob job) throws IOException - { - Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); - - final Map> intervalToSegments = new HashMap<>(); - IndexGeneratorJob - .getPublishedSegments(config) - .forEach(segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) - .add(segment)); - - final Map> intervalToIndexFiles = new HashMap<>(); - int segmentNum = 0; - for (DateTime currTime = interval.getStart(); currTime.isBefore(interval.getEnd()); currTime = currTime.plusDays(1)) { - Integer[][] shardInfo = shardInfoForEachSegment[segmentNum++]; - File segmentOutputFolder = new File( - StringUtils.format( - "%s/%s/%s_%s/%s", - config.getSchema().getIOConfig().getSegmentOutputPath(), - config.getSchema().getDataSchema().getDataSource(), - currTime.toString(), - currTime.plusDays(1).toString(), - config.getSchema().getTuningConfig().getVersion() - ) - ); - Assert.assertTrue(segmentOutputFolder.exists()); - Assert.assertEquals(shardInfo.length, segmentOutputFolder.list().length); - - for (int partitionNum = 0; partitionNum < shardInfo.length; ++partitionNum) { - File individualSegmentFolder = new File(segmentOutputFolder, Integer.toString(partitionNum)); - Assert.assertTrue(individualSegmentFolder.exists()); - - File indexZip = new File(individualSegmentFolder, "index.zip"); - Assert.assertTrue(indexZip.exists()); - - intervalToIndexFiles.computeIfAbsent(new Interval(currTime, currTime.plusDays(1)), k -> new ArrayList<>()) - .add(indexZip); - } - } - - Assert.assertEquals(intervalToSegments.size(), intervalToIndexFiles.size()); - - segmentNum = 0; - for (Entry> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final List segments = entry.getValue(); - final List indexFiles = intervalToIndexFiles.get(interval); - Collections.sort(segments); - indexFiles.sort(Comparator.comparing(File::getAbsolutePath)); - - Assert.assertNotNull(indexFiles); - Assert.assertEquals(segments.size(), indexFiles.size()); - Integer[][] shardInfo = shardInfoForEachSegment[segmentNum++]; - - int rowCount = 0; - for (int i = 0; i < segments.size(); i++) { - final DataSegment dataSegment = segments.get(i); - final File indexZip = indexFiles.get(i); - Assert.assertEquals(config.getSchema().getTuningConfig().getVersion(), dataSegment.getVersion()); - Assert.assertEquals("local", dataSegment.getLoadSpec().get("type")); - Assert.assertEquals(indexZip.getCanonicalPath(), dataSegment.getLoadSpec().get("path")); - Assert.assertEquals(Integer.valueOf(9), dataSegment.getBinaryVersion()); - - Assert.assertEquals(dataSourceName, dataSegment.getDataSource()); - Assert.assertEquals(1, dataSegment.getDimensions().size()); - String[] dimensions = dataSegment.getDimensions().toArray(new String[0]); - Arrays.sort(dimensions); - Assert.assertEquals("host", dimensions[0]); - Assert.assertEquals("visited_num", dataSegment.getMetrics().get(0)); - Assert.assertEquals("unique_hosts", dataSegment.getMetrics().get(1)); - - Integer[] hashShardInfo = shardInfo[i]; - HashBasedNumberedShardSpec spec = (HashBasedNumberedShardSpec) dataSegment.getShardSpec(); - Assert.assertEquals((int) hashShardInfo[0], spec.getPartitionNum()); - Assert.assertEquals((int) hashShardInfo[1], spec.getPartitions()); - - File dir = Files.createTempDir(); - - unzip(indexZip, dir); - - QueryableIndex index = HadoopDruidIndexerConfig.INDEX_IO.loadIndex(dir); - QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); - - try (RowIterator rowIt = adapter.getRows()) { - while (rowIt.moveToNext()) { - rowCount++; - Assert.assertEquals(2, rowIt.getPointer().getNumMetrics()); - } - } - } - Assert.assertEquals(rowCount, data.size()); - } - } - - private Map> loadShardSpecs( - Integer[][][] shardInfoForEachShard - ) - { - Map> shardSpecs = new TreeMap<>(DateTimeComparator.getInstance()); - int shardCount = 0; - int segmentNum = 0; - for (Interval segmentGranularity : config.getSegmentGranularIntervals().get()) { - List specs = new ArrayList<>(); - for (Integer[] shardInfo : shardInfoForEachShard[segmentNum++]) { - specs.add(new HashBasedNumberedShardSpec(shardInfo[0], shardInfo[1], null, HadoopDruidIndexerConfig.JSON_MAPPER)); - } - List actualSpecs = Lists.newArrayListWithExpectedSize(specs.size()); - for (ShardSpec spec : specs) { - actualSpecs.add(new HadoopyShardSpec(spec, shardCount++)); - } - - shardSpecs.put(segmentGranularity.getStartMillis(), actualSpecs); - } - - return shardSpecs; - } - - private void unzip(File zip, File outDir) - { - try { - long size = 0L; - final byte[] buffer = new byte[1 << 13]; - try (ZipInputStream in = new ZipInputStream(new FileInputStream(zip))) { - for (ZipEntry entry = in.getNextEntry(); entry != null; entry = in.getNextEntry()) { - final String fileName = entry.getName(); - try (final OutputStream out = new BufferedOutputStream( - new FileOutputStream( - outDir.getAbsolutePath() - + File.separator - + fileName - ), 1 << 13 - )) { - for (int len = in.read(buffer); len >= 0; len = in.read(buffer)) { - if (len == 0) { - continue; - } - size += len; - out.write(buffer, 0, len); - } - out.flush(); - } - } - } - } - catch (IOException | RuntimeException exception) { - } - } -} diff --git a/extensions-core/orc-extensions/example/TestOrcFile.testDate1900.orc b/extensions-core/orc-extensions/example/TestOrcFile.testDate1900.orc new file mode 100644 index 0000000000000000000000000000000000000000..f51ffdbd03a43fadbedce302ffa8e5967a30ad59 GIT binary patch literal 30941 zcmeI51yq!6yYFY1p=;6DUCQt4DAB&1V9K%}IV7?2VLBo!ov29;7` zK;jJWhPdA^&v(|^XK&V0x!3*F%`9hK*Zu$fuj|p&kz)n`u+RxnF=92e&IS)ywE$57 zz+C*2r4<0gpdEwhZ|Y0^$;CI|xXKhhSJ^aOw}{T`T6|~omgRWf_uxH&b-{8>HEA1~ z&-69|wFhE}mp3-YR-3*LqL2e@k#{G>0s#O3LVV;S#4#{7BS4VXqbSISQDC9v17D&n zgOG4rb^e(|N#XkWa*vfFZn=GcBF8k-doL zdUxG5y@)$fL5*g@Y-!l5WNh)8$2NB;-b9edZ|fjsv@i|6|Hacd*q)BT9gf!s#8N^o zFTR2t^E=_!A-6rF;MF-&S@bm+)o%-ojN|2L-Mvko47YE_djj%;<7!FE5*~$xnnBM# zvD4-CfD`Gb*t$Q*$iW!>iNGkBX#@~*G;>r!;#^ETPG50A)`8U;Kx930LipKgJ&8EN z4}Ix}b2i@4z^sYWO?3>KdXUR+A0nqhTO;_DEoBn?-d)^39*F#SJCstA;l=B$)66L` z-y!0z=u-gHlaEl@AS(1OCe%6E0<+B2jF|PQv7zlc-e9srY_v`W?`8(T~E+{q`PMBnLfU%6#gJ=NC}oggCw= zLp@Luj++C(jDujty{EujCFD&0Y!rNB0;3go8R!B)ReBJL{L`momkbxoS~Ll^C&T=& zNBEvAAnyWL9lKmLHwr}GkeJJhe9V43g@1Hz*>4MakB;JQYLb}R3bRU_&O9-@yvE57 z`$+1iHh9-kUK20c7%uXLiK(T2@(=rzYRz9P)v8gHj zn?X&<+%uz0y~EFqlzPb{^&9#_pKH@@zy`m@+Zc4`%6AfazFz#A#Pu=0^dpTCk7*^Y zkGauKx98W0)iD7HPSYwz9~;)4&OU0mo%0*M$VX!B7BQ|=+)K5_t%=4ph|rP-$q%mR zWeYLJU77X%o9rU4WX|(t6YZ1rspwal5K*??-Hf!~L#H>yS{5SjOMNG4nRE2-7dZ%W z*<+mMYKHq9y~95UquaYIwJC!9rnA3q(=i9L(F3use#F?Dz5E2C@`TV+mg{Snf+)u7 ztm6|R4OyOyFqwLcJXeyZ9aI+KfI(MG3Jiu|^7sxB-up%dFhV_7bjSMpy|Y}mW(Yht zz&_sLphnp3O|A?$Zr>${{cU#sOT+`S5^)cx_{i!k8ImlS*V^}Icq|!d+x4zI%=9Mo z+2q8Fq-)o1e3;-(WQ(a(UHv$vLjw7~>(Zb?PeJK*rGo2RQ>r!9$&@SfHG|rp^Am=7 z{2!yOT#6plP0LRgd@KEwOzKS(jt!GlYB!Zwa^0Ou#Er7~Oyjpc4RQE)F4HYIZ$?hM zwY?k5^Nr1HLAm*jKn}6t4UQ-jl6>H$&9Bj%3dt~hCj-I*qD6b?| z@w1WHTrcYZLv?PF6ji^xPC@case=QUn^s~}qorD-5ztDd1ei;aa3XT znfkbTx<}5?zK$)dOMI|WV=FrAi&4{i)h3ke<(lRXREIfr``YVU2~$!B>U3L?(9imQ zgD(3~fZSHw2QmM zR__dD;jdJNiQIH{?rr1j?U_ieX%L~>7`@e-HW_{6m>fIO$sS_7h zXD}(s^o{wIv2lZjni{N9xti zpsBI$PT@)sPiL3aHttod#yUqI7m=M-=heK)G1gb;QsW_hm5M(cGZ@P5Q;@yv z?%=*H;;i8ZCv{%YTk9*GZ2DA>URMBhm)M=(SsSC7h?ID#@0AO^V~g)yUAIM!f3N#B z?USoKyS1h>iKYvyjpLtuaBaLxy|%>H=eMWEs2U?hbecu(inw$FhP)#vaQJfh)i%(U zyd$xVXR+2@)tlOIp>R^V+xIo=yKJv$m2W;M_ihup@&i_QSkQmvU`P?V6VbBi7Vw44 ze?@8AP01Iy4fPs0Xfp|(4nBUB)h=-SC zNWKP(e?(Vz#TbW^yAPAe&}D=$)Pt>Dai79T-wh`-&~VGKgfd{=R%%mExn>q7ZxF6q zkNw4!PI!jlR&n`j?kiz_-gx&n2xDf1GKxfIsDmEFBoPp`V#teL!asia&5nvhn?iSh z`&pjfu$D`nu(*PW`@{WZYTN;4%3_y1_LuT*IF{n3ooYB%SN+J4-`=H7VeVP}vRM=3 zYtvooz7IYj$h|h%*XG==Xi2qO`9>EaNc%CBmaS8HhhKr2MH$|(yh(7!=TRhash<8s|ToLA?J<8Nu;VvEW z8_&OAt^UYf@i2a8MT}-pRs87(cNLeF10{kqPE0v^88DHoFjK%mAaOnu$yi+4S7kZ8 ze7}QO_xY>$?y!IwV&jb8Y#|6z1!4!r_^tR;p5WNdYrp$;E1l#$;^rn(om&RTk;dP&WF?<4|WHJ8S*v zw0cYWQ5WA~L9UeK$=F}y@R8NNs{FAFn&`C8ep4axpY@u;mf@}Bheun!>npoi{_86( z(^*G4LuMTNL*M;;T3pt5R}bqxO$9jp&8b7pmgALOsl$Ndp#YbsE!VdSmwsgX9}gXy z9d8{2)=om6MAesJ3N*OnT!Bf1vXwaBsLe{VyV*;g2L8-twZeqS2iL`m)i`8qbRw>^ zBr4g652R|iKPAdsaTm`%8C1PC`}hu)p?tvFyyP>z%V(6ZeKWJX?u(^KS;@AuiPV8j zVtH+erEb}?5og`QUWQF<4 z!p-w7>8&?zi=w6(Ufly1uogR&^(V1effr)r4QaZN>uc>E*jr*J$f-g5XY(D%Y z24bF%f!@6WSLO*G-EDJ0WK4hScx|@#aC%dyWr^wtR(BA$PMW5eQTb*SfN;(JWiCuDtX^v0pu^;ko* zj(;x-8hvbI{0hh1*}y^f4w{>l5N)N(kf$G<)v>gRa9_D$Iux_&S5ysltv-3q}Gcj32C{eK|*Xl4N>eP5MjtrVQ zAGZ=(KoW#vPQWEGQ9F7O_tXQsePYqM+ze>g10Txx?+UV7VD-{6?Z7Y4;8z z-Mk*OXL44*A-Vq>z1h5Wb5DYAf~_=r^^5D|s2obISlYrWcvIJ~3a)e56b|>8+CL5% zgdUizJWd$AY-^*%-BVJYUK&mKK_%avjRI}*0&NP3JmDP$E9xjxUmN$%H_OpgeB@+S zf@PiHX5pJeDb)>v6hvv=?JU6Ql!gW&>?*;_ZnmR~Q@3DsHyvqv`>v!m-4#avZW)80 zot|)4z!uw}x~7^O+YrOog}~8jVRV$^`QWqXnYtl#&pX8pNA(L{p`kk0Y6vHWTu3%eXAs_<3b~Bv1PP-;K#OQ%m79a@x?o?ceP-TJkXh!Gm_&%%ZH1-JYJI(_ zbVn%?JT4E0;2|rI?r9_IogS&d(*2E37=lsdb#qfEt^h8?86RM zQM_S!O$A-v@8xKBFqrQ()NNyxZ*jORjZ95Bvb}a}%KLDawS2d43U1OE_pRy8tGfbd zDV?hC%pVXWS_F%Tb{QGN3H812JARg%fPE;o=5<4OL46n#1aU^ za+MBNk468KlLN(lW==NT3bhNCdym$YlLy7#hg0LUGLwfm@}kz6KW)ZXai!pAz|npn zN9f#^vCq%h#{ay{#Nhsq3QT+pRX-BX8TPNQ-9$r^U2)21uKLmR5jyrRlZ3)Ju3#p%;Y2r-)EP>)KGe% z{LMO@SUL6jaC*MRj&cHZys#LH$sLt@pPl%blC9Os-aMX&7ei%NKg{4~SJQrYE#7(X z%H-3I*8zhZlLa0xHPg3ZYghPIi_C_gl7({jl)JXlG*-AW>DXlxm@P1v9+=B^htEG+ zj|+aBe@Q|oLDIfEsQS@*LQIB|W}h`Z8GrocR9Y!|H#B~6qnm1I0*_~+t~|VvcW3eVLHV68jeog9ita$q2tx#3CqpknkKHY=-m^G5J#;L!yLF86?H@)y z){oIj%sQ#p#ZRMNyh1*IriQI}W}kbm05UGJiVsDM&G6j1Y2L?lL^S=qY5m8t?KRK0 zi$VrNUl!7CycW9mZq&Nz#?qJ7mj=mS9GZ+i;0N4TC|P}GkXYhSV>D87`0NIGiKmCL zN6FALvrPThpI=at9XS|I>Iq;Qq?x?H!M;)5ZP4cYF=O%ts?iIQS5{v-e%yLZ`!PxK z1&xoT&(1r%Nc@j+O)rRitZeaI1?%QhI_^`?@+#KjCc7Dx!Bxr(ecMMSvK;F{AMPlY z&1o37AVR6?u|@9aJI^{#yaQ#K`u3cUgk8QJ>xi`J@3;YQqoZ!GNPXY_?7g6t4aR^4 z)}zk;NFK1E0ag@pYi}NeLB(L^%T6rIe!WejD6Hu{2Ko~dUuO3<<>jy{UJNn|;>(t(0pPRo=Kp-k~&CT4}Rp06eb7zSykZAypVx z4UHd0VaQi|GZNlrT%6V8Y}a|-o@pR;pnI!5d5=P9(X zN;ydsg0OTdWFJSzhj=}fWlvNYW3PFkPq~m&jURALd?A0U=LW^&NPZ4|8LBi|}>nM(3Q-MuHt z_fVmOj(+7L`!X)uc~dK_O#z5k&}A~e;qv|b=`9N2)ey~dk+5IuTqGnpzaO+XzaJd@ zWk1M&em`hNRfVZGZk!f{scvnV5a{x5Vj?ZFuZ0gO|eQq^QKEyZ);=9 z-0P+D`#~2@OQ+`(I(h{K?B965|>Yr58 zfl?pO?FJ9*KI}|6e+`)&SA12a>51?E0{w^Oqn(kguj!H!uqUnkcyrR|Vjmg$I^SfL zUJEqK=ou5Msq&&+f>-wqtiZ}O)-y6&8$|QmJXn?%-{Bv-+Iy|=bGyeynjpU_8TY!l zCI(93^c zO+BCP*3|7%_G}ogFMCYA`O;J+;!Wm~@9Xl-c`6F{bdykk-14eT3Q`5xBqghFCz+Z_ zyGJL22sH9-r-9DUW^YRS1uFV>5H-(?p+i@B{1#`jq2DSs52CYw7`1Guw&X&;IR>nY z9JRU}=1uRhwiH4)(T`^z9gSokq)VaG=M-!H5(lx*$3aun7L&mLWF##AB@&7|R?lwX4ZIbobw0&;pS-FYMuKGyL z+jQA;LHkhJImas14%vaD}IpZF_j1rx>JioY_(BCo?jvTdcq`XgT%PdS}Bfm@7O za{JnUG!n+1kA(5(BH=XSzY+;Io%biVrqxb^+t1POw|!s#ii?XzK=E0`I*oH#L(@|k zzi;y-3#Z-on|zMHzuc7?|7|ZgwLbadFQ)u7j}LwJKixdpMXsWYp2j}^>7ju2tfSgg zWF(b3+C84jIx@Qp0HAS6W=Q5sK9{VPY?N%5d?z_9Ie{i5`BidD@&F2k;y{U^)KDfU zCz>Sm3iK*e4SF4F47G;dg1SR}Ws%zLFjD*m5X);1Fg&O3=#kN>e+bEd^xe2iLxlDh55A1LBaJCo^Ey6$(HpLwPcCNy;^q5Cx+i zE3_|rW*t;0#t;}-BE7uVhfl343jpc|Y3=pkQ*)dRN9NIYgicQ=((^a}KyZ59OGSTk z2?j94l?0=7eJ`jt3WD$o;u!{02l-8S?SvD7g#O_-cs| zX;?|1-Q+5W5xEc};6QE12Jjw|k66x`6Pd?C7ElJF2(!%7l-r^rGeZcv16on6D^Msd zSHmu;5fY__rihXUl3x{2z8!}41)Yh-_NfgE2tf1`bdf`j>9Ej{h~$KlkRU0E3LZvX z{a{;_HDt4PVR!AC7J%=+V3dMt6v%cUl9jNX=;}?l!RAOXL5@<~93wOl_EEJfXHSJMrTo7%_Z_I(05;Nd%Zu@9nS)d>} z7dr-L3x;#T7LZc^fKt70aGlR9zu={76z6e?>mIn}!)Aql@Q8qf_#cF&jYd@zr0^4v z5WlM2ONKKjq=f;-490X-Cy58E>qIXDm|`EZLx#;?%5VuvSzsRVc?G8LYB-zTe8oc6 zg7}W8r;l-P`YhyUoWjCrA(HWEorSw~`wqeR12ffN zx_)u)){zy;KMo+s2>pSB0Kg~ax8wqMKd|vpq9KM_TbJBgGJG!eXzgY5&01D`J(bdS)8;)rM=zr3!WvuiZ+(Z_Jf3N9aOiJkVHucqCSlUf68eynO#9IH}n*cj(v&m%M_ z<4}+{tjt<$)|mJUa({asxy%28+)Ms0a$jfv1-U={1-W1U&ms4?Q9Ja~kr>0m%xY5! zM-ak*s-{HPUc_lMNaKrYCNO(hsY`r7RBo-;Ae??;X?+F`@kb=zd#LzLeHd1i;H^dbJq_m>UnMu!>|AMSNkWIE70(pKF z0>1p}dP}u?@Z)m#&~7@psRd`(hekL{>}Sf!4&|eVPcn&4rID%r16VQBsWjq;ltx~j zsU$)l4q#6hPiH-n4`5}ir^9WhKV%fAn7tqeg;TNS-q-J-FQ{jCzP5m_rz#7dM)ju! z036wz$iu9|V697LR4L>D?>`g}_{@m{!huvkWYbO+5VJQpCklvw^@#%Vi9GB?0ZG9O zEvT0b*#;3N2n_Kj0Dl)iPRyp#KbuYQJb0int2ACmg6NpOjiZh0FZBVS^ZLiBS2X$pN!xplWXT%#~<#X3_Y~mDupj6G@}G zovw1k_2dugq~;SKof>gHXNanBwK22ylvORVn9bZo3uibYZaJP?*_+WU@vSi-DY>It z>REEH#!Lv?YNRZt#M#4eyX3orfr;K+o0)paCx@Fr)hg8_3x-Hi3e}Ie=~YIx=Bl-p zUbmXOJA}UmIgK%Xs${LT?f+VVh@~03EP*sGqsA)7JL{q&CR>=gSY;rxABI!=$lQg9vg0!JH5a#`#E|Fj=-Rm5-P; z6=ak%^wp8&p&Fu?O|GP9Gnb;;bW=#pum!-e;i}!z7zFLHoXtUE6KyC-6~Ph)uO)Ah zPPdWLTA*uI+_x2lVN;V$w_;AgIZ@(BlP@vK5^yz>Mbs0x-M&m}$*wo6@tp4HlJKm| zt&Y5p5v-MD{j(L`j4`vi8Au&lRXVqzpq+C{MX8N2PFRQAsX&&az+qG4SwYjV5_)lB zj1FsNzW-*Ppc0w=g)w$2!H2Gj%Te6lmkA{MgVh zxTc{}o{=ciz?OGMmt`Y}zeKuVq@brrOP)gy6Eb2v`7F%vS>nrUjvnR0FfF6)BG1pt zt0Pt7MPwe}X7OSXkI_H{N7d+ooN(!<1WtJi<9e5?bxNu;^+NOw^JuE|Bpfn52k)&u z$A!x|4VR7QPgdVR7cb@UK=aLKr1^4ndby!_1iNYY@t8|n#k^*8;Pdl@5hGhifXnlNmxc)* zRX3?VRyS2sY>b|FWMrOqWFQ7;zeY-q>-)TL+4%~%k0vD+1)`ZZzb{I~>Q$=&^!Mbj zT^jD2vd;(^5?3q)wTL#kd7>@NM)i%PW~2*99ZVKX7QC&0FVF26@#?nQXURHO)w(-g z&Z}#aos#uj+s&UESnIs{*WjiT15Gc9wu2;m{zHBY66XzI9pZt6Z<{0(iDDp+5D!^> zUsvJ#j+^bjdoATRZ*}60+O6}SN%oib`}+6&7};oQ01z=e5S&hrUn?WyM|!=LyC}r( zJ@|eOZj6j#6W)I`Mf#OO{a^Yq)UR}Y@P6pBnGqPOk%TGNq<)@4aTh1}4Km%&SC3In z&WO+2=c9;MXM9Blz)Cf0b0?r)(YUNdHxq^U&n#q`|{%2sO{3N zZtgL}KXNCW8YWIi)vJ{N>NAtY$=GY?SoFj;cziSP`>7|fsqom;=MS&Oi6?OvX|Pyb z;SoBsGS=<>*zup93V5}8_(Aw@P91NZxe$@g#N88K6|moR>Qp=~57-X?up{%OzW`5- zze62z)R}xwO4c5{V2ofQQQ4=ip>>fP^wEil@=R_AT}~S$`up0LKyWTr_x9W3EfXYb z(nhsznggYuP5O7Dk48YG>7NDq)@}ybuz(>CIOxs1G^Bn_V|xBfV_MoVPSTi3Uug=3 z9JO2s5jqGW5L7{|Z&%?jZF%RW;Z6R3g5wnJw|!NDY$m~aibVg4<9wI`LW)a%&trb8 zc}W2t-lu26kQK*Oh3phr0Z9Ro3OOk`xli#zKyBoQU5!BJHOafHsm(9Ne<^G^Sba2| z#k`uTH$?hpT5?+1asxcyle$=DN1-Nm2y}5En)Gt`)Omhn?hF989YTe?QmCF1zWf zs&!MQ{_iuM4?<89YrqA(xu_Jms1&)V6uGDrxu_KRUt1}10dFEN;LYEPf*0@xSugpo z`C~8O%|BNuashAt8&-;3z?*Z~Lok9Nt|%o@a4?U$M`>X8n6TQjq`kCHP{5$kUD(!fU zj|lN`0|5sg0?__~-t-m#A#*WE)#K+*ALM`_yI>1!PDb;wFp_85%-$?TB`~Dyp+u;a z8AR9#r*lKEzq>mxq?o~_%Z==p@jHwmj|!ceYVp5DDWrJNU2D7yqxr+tN{-pD>1ula z3z6EFGGb(xir<@q;2@iW95=S~BM*${{&YYjX#?LM01&9Ona{N3SAn&`;oqBhxEl$~ ziw^p`@0z~#d`ylTGT_dehN}=6jN~@IBOHQjgeXSA`a2m!=70-;a{+KJ0L}%#xd1r- zY(eb>!1ovcFhmq_@Kr}49C3VH+fn&=#dc$z$Nvkwb>!!)wwDfj-~ZsBwb^AmqgiNO#2? zQ~~Y3q%%KLoSvU4PTIdoaR$da3%DSgu=vtxWl?L5#JXdqvqCm{lSj<#jDx%uM0x`O`T7{N^+46Geq7`V@VWDc>p=v47 zn4kRX#BaT=P|Mzye2B*z2*fc5fbR0~8lw~*x``X3tbfTPq(M%R!vJXCEX@&k9_edd zn2l|h9^xIP2k~f~pq)z05NYyo3sxnTdIo;vZ)V~Q!Y-+HHjl%7E)OOU;WfcXQK`R_ z8x|Zaf9Hb@YC|1@6yg0)zeMPL6#mH*4=C*3H8qhe(eL-^GoMW%pXg6vr>5&%d^w1{ zFKiQtcDghe52zL#ep_mc5HX+13q`K!WEx=uOM-nLX&~fPW&7#!3g*}=4*DpE3tV46 z(j5R~0RSIM@BW=PY_!P4OOQ7WO<@RTM@i3sOYPh2)xR0$dr?_)(Z(zGZ^=>@ZM-hp zcwMydx@hC|KfaCE1)RBnGk-G*Uci|PIP+5*uZzl>|KV-CF5t{5oH^NzpR_GGg){A@ z5U>nH2I7ttX>OI5h_sH5nAw08l@fu@GZ@9xa1xDJ4(Nvs+CDV1+s`cYk-b2SQHcwp z6h|tY&c?lSyFu!TCq*(j%WDlm9-3!EmxiE`P-Kq~UFprnMYJTJlg1^$jxX_41{p|} zgSVJ*v93Y&G%WrM_1w4$l{t;r5--S@uVo|+0%G7e*@1v zJT0y{?gal?TqE&s;hBV|CPysH8=I|D?1iBqasU@n&LBRMGjLBDv7=!GX+e_3v;Zu> zJ3z_9w25P!2WWVY1((tG(2C@(T9as0f@Sq%W6^>batxj{mbFUEa=W^h4%-U^ zqLG^i2ED*Bo;s{^DUna3Eo6Czj>8KBfN&n$&?mz_Ht98h*RpL=@R8>sPiA{7%^ftk zDV_`~g@*ol`8=WM`l|+Tc(3?xS0CHM(b)iis*0wRTYL{ILy`SDe5AJAzw=)}my4>I zi>jH6s+o(bnSZ*A=>>H8p9)=?PoPVAt0%GIABowPLabP$7UYHNZx{kE+3H? z#gTX=cekJrCv6Uba7 z=Wf*+Ei=dON1&G$eAFGm+6>f|&=|J`K9Q>Q;BB7ngG%ww0BEzed6m%VB z{dTr0yfMAa_awLB-N;7sSNRjJ%nYO5ca04Im~>;Ju-8<&g1Lx^3GpsnjeNYq$!yZ& zSca2wyw3?)<7!5QX3f^{Cng;t;783)_+D4$U)|061EA1zf(%hQs`m-&+avzqC!sdd zs^+xi&EW8N7;~T#07MRxHbAQe8(;fDxUA81?nJn*-zBafPvx^)1|17-pmGBMMt&cM zJhzc#1;jsyByK*e+f#Fv7sW0YJmqgHNf$ijf~Q>Ylnb76!BhVA*P>i>D!b@Z_OHJj z?4ncIf4A47Ty!e?-}YLR3!ZYyQ%*WUobVJa;;@;w1jdvH+#dX$Y%%hiT z8SS*)`}-g9BCg)ZQWj)$c<54N(2nwHOS1j#yP#Ld?j0w$@VE==l|7v`9B&MYJ|#1< z4n0l&y=3Fbf8EU^_*9l?Y5j?tVE;8Y!9fAc4K7(Oq-ayZV9Z$gjR?`&X~fZyZ4<1^ zwr;mi6@>L26jmxK3iag2KA3(3CI;B2z|lu}Tk-QPx(ejduYsY?KKq9zyaunqQaRxT zMk&a*MZpB!i$a8ue+Atq_J*jLHZ~2vH-9lT$EEo!jawBUfOX5a?LluiNef=nCIlh6 z3M(cd#h@yiwjwrYNFzwc{kfXPFBI1{nIG%Q8p!^=T;0rIua&9?DwH-QVC@NxRtml~ zNJ7z;H4nC`MQ=tn8LT|S9gskXK$x#TzSbsfVH%5ii#i&JGQ<^ROyfTHv&tep`aLkp zcHyHFCyWXP09*$G=FCZ#eY)5>-nMe`hG|mAFCObdkIVLhl;dqzWm5+zfNz;rd>4e$Ml})U{+;AP`(E5)AB6HI2Oo z_gOU{(ZEFpTucb+7YRA=q!JPKyR4*+w{Gy@ik^o1!DztUtho54=%ug;=psRsQYbnJ z1@?VbB{=rG{;0U65Hxsj$XriSG;pIi@M`3^P~gQ#h=7+8!N%@*0EPROL4ntx!o^HR zdyPH;1)jRk3Ot3Zg2LZ-15YudL1sFh5L9VU`f1`%dz#Y-V9Q}B>P*1dXg+$`E_Mn? z59u*0-hwojzpNA;=CXHqtD{=iDe(~tp9cYf;*Xjo(T*4jO=+Kse;L9#3spj{mDp(EJgjE<{*pVE z#+VAIH-+n?L3!6C!myspxG2^5A`iY?>D>tYduh4luL?IG!}30H3RaWL!IvcasY4g< z&S6R<%pX;BsV0j;5sxPtzA*&qPSr$0Io1#r?P~GWF5MQD3^E%d%vPhNe{dnrErE9trY5Q z5_)OU)S3`PFPc%D$qT^|^ip-e!gW9kpQn)cl09oL=a)ahy4VQkSai0d394 zMD9HHu5aC3Y>VzDDpvgXalGICakx2;k;Gh>+W%Q2BPjt7*{{d&b&O27ElJ#=r@H+p>a@xvi&B`@ zC>i@1wcY<%2GFFN5uS^wCL1%;9pmvkt@5bX-LLo26~1SZHFG=8cidN7IgD&{YM$$B zXZAlgoYDDIb+I6G! ztS(wTZo^yx7h}C|@0_L|%yjdMavJ6_xtOtjt8Jh`TG-a}285h`YZ7cv%{5-AUm2-i zuFhE&2}5pRV{Ea8Gja zNX3Y?d!x{%m)#oE;K_2N%l1d2O+Wgz6Os!;?uM@HO;uXPFg6A>Jh?9Hwc!&_;7U(O zoMcW;`^gJq&kS2XAyt#%j`Nc)#!j(6l3qh?au?EKk3)sc=uH`3BgAu0?;?Waf<2aX z@x4#OC$9|=JI%vfobN(Q5!V+^#vutBj=?Rjv-2GzYaSL2yHCwaQFNIx>N3mh+w~#t z%S?1{FNpcaJCnF9a1sX4ztt@Dk8mb6$5F`7e;89Eilcb(Sd0Aow5;P{M zADv_{t$EWSWt_0Z0ciy4^rtAlJ$Z3X3VCK-cg^9F1nGsdgTp+4-06VG>$bk37wPeV*#bbA=iXwB*=Ky#fv80mT;tJS=cb-quB z)!D38HQns^0<_W&`I+acJ*tl#lUmoc%n@}WExBq-K?zfOWlhRu=!p{r<(<@9Nk!|L zb4w8elAEgZO92vJblh6()&n8QmFD-Q;f%h-($o)API4f|pUQ}yPZ76jLMxcKFEWqu zrJ0Ig!dw348PeBJs9Y_ZYFmu)Ef}EGZ@VZ@ufAssYQRS1u?ra%HKu zn=&wf_o$93x?Ny5?6?h=YOU{azFb9w&{bR8PgR_s-uFE!C=sHb@3{6UrTY1eM zvEj#sp}Dau10TE5$!$)ato0^GAG|a=+1+!oq@VydGoQoSydCM1nM-)Vnl`IJhT^F)db3>BbPF|R8$m%caEygNJlej6mGWu<{o3<}N z^tH~dWQiN*M9%p|OO!-+-c{2*k>g$|G#rXF{5(_I)9XTD=A3nFNn~zxd7zuRG(bqJ zT5`2iI^Z!(|CJm$D_0G|FZ?wq}9SrvfWeYMzPJ-vl8DPR4h1nOi{Q`>a?-C&&+MrO0H&kEQ#)nuC!%rBaJqjE7l`C7RMnl zf#pjG7>^Vhp+XMMS7U0jwsGkr)7~OoSgQ8eJxI!%Z4}b1!9%Cwo(c`oc-;?< zA||cGJ|ldLXvhDn1@#e4D(z;j=P7P&8g?myBJu_|{nsaG2%1@rKC??(_CCsKbRMSVJKC&xVe#v;K zXyUIG7k)EhY*O$f-FEziNv^C`x1Q8{`Klq7XSc$?O`fK0Q5H$>ys2b2xv4-L%ZVe) zAPpmx5oGq*^dw7VKGGc)JFO^7nCi9}dUI6n&W+>HlF%^y`lVIkNBp79-F&8eF;DQ# zM>Riwz4rLXrAOM`T$y=p9b!i>J*Lg$5;b#kYPdTc9u(p(XjXL1(C1W{;mb#-^i%4t zsdhYeEFX9o;8^mq7AD#pCTd-mSD`N%7ss?m}B3rt8XQcI+^P2VK_7 zGJxOa3Nk9%mZJT)mt5y2R}$RkCMyOLwzXP~IM!Pi-CQc{=2m95ih2h;Z2sk?E%l0> z$rZ^hkDXQzyY!0lOBv%EiS9eCJ4QQ8JCNDkBTqDMs*MU0DL#^B@IcX2gtxBM+k3U?p!m~> z=Du)Ka(ajR)igtb`VBTE-X}_|7W{uX191+YfzF@!#YTxkZdci@#tnY0sWw{kA6z(9 zF;2aWDB290BTrlq_ZY+4t_sA=IgqUrh%QNfg}2a z{F@9Djjz)}WD^eS6N-i^27O=6V%*kY>pz`>dRMWm_Y|S)ab6aDA~iTC}(IwjGKUm1)>HOt>exJH;uvFE=! zPl>^yP>HK4s6;q$4ma&QhgnA3dqc~IJ}nZP29pmXT27doX>h%Nsvn8tn)H!F!_x5? zAIJGHe{FQRQ#wUfPok{SX4`c8GYNRwE{H#mzoX@{{DSby8CtJwwJ)mir;?g`v`ht_ zGKN~p^;(n^w&P~CaoA+EBW1Ii33W4R3)Dm{%bYqe=fec8bZlB|dtN}}JEBFP5*=Z| zj>&dQ`h#Em48I(4PU~DZv|I8S_;R+aRsGJ+Jds-y_xe+#WyOj~e)Rj8*~C5!GT2Ps zf^28?&U>nB`Z+z7cWFz4S~U|pBKg}=_>VSW1WouKzbVVVT9p)(R!P|;g(-MHm-ohe znkHqMCY5N4;5T_wnm}?_vzxW`6G3^rloX}{FKc1tf>%jn@=FA$NMqlnC`-yH8^enCDZo-Vel9kI3P=Z+4J z7cI|U#B2M|zo-?tnB|x+GIeFSqx&c#q{G4>tN!(+Jc4UIhCTSJ_40y+56W^KT&yXa zEOn4U2E8tCa*3R*>1P@wy2YKXiN4j)&18yq3%NLX%K9YI)_M`%$dQc;iV)SyfK&)a z<~dlI&j~bCTE?fedKtwJoEG(QxGq2bMWx}>Bl_*hmf03F{fT1~Z-8;}^m2k;ahhI< zXpz&3?Fe08)SbbZzKGs8=C|y*XD1B%AFLy`X3(7xDP?JO&P}p(->_NkloT!FW-oEr zjkgaBShH1Im!)*xX3bt{7(g17hJ7uoe{-83Gp1Jgqw#&>NE2@%;W|BiB#DmmJ?k&B zeMqS70`aX?Ul-R}=rS?g!V&e=yXN$(`4vl#?s=@etCIlc(OJ8#D()@+JD)qZ2kbU| z2G`G4EULS!yA2a<*U)ceO0Efc5K2j%%Z48JM8mb0@>UJS>Ph;-m9zk5kpYDMrwQ3!Xl1 za6{?N7aKn2=qoBO>mT-opT=NU-HPL5SJCu87jD}k)}LNe?a{)~pXT^l{qa(0;Uw?O zQ=?W1@eJ8Jigin|YLh49>Dgr>SWK{)?;Fd!yE_^(7v}vi^_Zwugt+B9ul$g?h~PK{ z^@b}96nx<)9??lMIAHRL=v`I87t7{0Cd|({eHc4+v32Tm?4{~ccRqGpDZ4a2 zHS_vn^psVZUMG>qrO%l&&o4%1S{3NEXKp>eM49R2sPCBB`rIg9r~1pQqZHd#y8RdU z@h-+1yu!!3l>hExm2FpC|0@i=S4ZBMPStE&tETIU5`RVOa@pnEN5TN2uCTIKq%NlB zg!TeOqcJu2Xoh*t7867}=)FWLz0`Gm+uoO8T@3BKarWhintsKqFLg1V&1>|`?R=d0A7`js_dV?8l&Ww}|UIN_JagH2&V_OdE}RX2vwQ$Sw^Q|3Oy-75FWCjVxP%(W+J)HG&X#J z^R3p4xA7MxH?^<5_383In@g&GnLDAzWT8yhU*|gT4~Y})K~xk;-m0_=N!0!Pup5>q zj%cxK2hgYK_9zG44Aq@vb8J!kl9)gtspxYvOn#C>wM8}7jG;9k(MUwE#{Xi(e- zy^@QY$8^4`z5HC(_YSKhGUP_ls`l*XJh68aw;yJIWgZ-*(!kB)JR%06KPTlAd7KmZoVxV>64k>1J`No-^f~rC-1E8y&jWF_Q!htCG(WXx z={`-w4 zP8pbFE%R7?BQZhY)|S1VI=DoICg-Dm=p<~^ICK&oIsAFhXhfZd zqtc@v8;GNN<#L3l-N(MZSieiLktO`r4t9dbg6`O}dHm{E?Gip!md1=!k01U#XvcZk z=0#tRuAjDUWI>+Qe0=*wtjvPo3+C}J21N9rs{==~GEZF+vM%&e?Nxq^CfRl9)8M8> z=eGgdX`lX{vv2a$orv6DVQpLv`PQB={a9QS@uaema6}60bQfboZGC*!IZvaw_uZ!p z@@^d+N9H#)Pa?9_=HlWjON3J#9FLEWeI(kkx4bpU$8|F-zySEC5_hNWWM4~GU%cKM zm!j)Lw#sEwPP6qh{duCr6t`|>&&rjJH_**n*iG`%Ers3_mw8|~Yg1NrqU?R|?Si6} zqwnW+=O!En>Zi)`gq$htQeEx7+71Mwr7w{1Q(l%civd)S1#*hww?_tsV(-ukt$OOY zzNM8IqDx$ma`QavT0>es)Qu%E!|gUUlqm5vSA8=maYL_SUb!M!;;XgCoX~cq-B!xr z8f!&{!~)jNaL9Ih;^t#XEQX|K>c5j&Q_?g|>-q~X`X#k!B5(K>!)*;I>lBUO=5@i9#V*J&2<<#Yel zlQ8t~Nf>_UBphV=pPYmXw(I>%gDR-Q?cnM6{c~UAq@A6b$Fsc?3w6#Vvxo6gh(>j5_<=2U|Vn61EE;0Vk>C z_R7vk!nV#tQ^HjOka^p3ajJ~T&X`o5jH^{`IsG6Ap7To4%q&bG_tmmD z19lbBWjIWE5Yv*$4!`7HUZqZ))oXlT2>MHOe-r*h?bZTT7~KAd#!~MlBI`KnVcVNc zgxeWx1|&z)s^FT}D`wQyxRo`pI&ZiXLR+et<}0)@4E2!LY0XQA5p}~uQ+H%P2|9K$ zE?dtzcVymi*tP)@s||vBhn_resb;AQ_D7?eP58W9vB}L@*sr~~=Iw2d9e(7YUgb9l z4?{+@nU&%qII+2e!rb4o% zBBMfsFNL4{D79F|XBkQo!QaXkfkyhBgzN$kIpC$VMlYK}pp4gp2_3o7f=OLG^k)+4 z*_u|is~*zgQ9>ypya%}k3SOiRLgB&R0C;h1SAYlpW&n86oZnq_W?Gh9DW1bP4W0u} zrcNykGcu5cpTNq=37pFs60I>n-tf=XE1c4aAwtmyO12M(W!e=xBb|P}3#vaaN^y7@ zc)VxHsra-HFOoO`c#-f}naiu%o^TOuSAvH?&8QSu<{#0Oz=CHuey|H978?X|PMBv< z*@Vm#$H%9Uh(>=QAOa0Cyf=T1wC#!QdV9*lcd@@G>Y1(u21hb)XlFTP-YGCTg=gCSGC0%L zQFl3W(dweXg^?;F)y!V2t7yu3%26ha0pwK5T?CKw^a_oY3oqZgR_0tIIPGQA&D5LA zT4>%lowFKB+kHY5&>Y7gXC0J-FS)*DBy{lYwLe|-8TaMr^O`#SPctQVMeR@C8Wov+ zyBwfi?DfW;>DK4tv$dbZkdrkTV%4&A3o)#lcN^DE8i{Q_%3h^4T68HNB69sezh=7- zJGaI!IZxp}n!WjUaP{_rpA??57jcp&j1z9cG#bF|C&T8a)P+-!LveJ7AwMA7S4|kF z%$^)`=vZKtb_}@@9zQJfuJSu6Y~r_}oaJ7p`>N2Ra=b+^R*PRFAFLv$Gr>~Nxo0kn zz@woUti%~Wa-xTkM;~L7AwF47@t~O4;ra>k%j_41)n3qV9}^swzE+dc^?)^(qH#FK znJIW!I}Z5Yv$9ljTH0IA0VRbhk}yFnE}JwNjx?(UwdZMNZ3YWmy4g-UtFu4UdiF40>x4_1CZW>CyiVFVHKpvhNa+%` zlpET|=e_tcrPA8d-apki!*Kxz-mc&O+)wv;3PG1yf+Ve!7QaHlO zMV<*cc6iELD+e_v-IYowwn_QibKyk3R%U+u1s@&V6x#d?qE_)vEq7*K5Fll3+Ftgg z_UB*163OCr#B@z%qMec(oG2-8$17_)F`sKV5y`6O{ zy%C$?sH;CEj97+}IA8j7PcpjeU5E}yPodeGAq^H%6avX7Vw(s)A@O_! zqcBaSpj2H9wYBdmqvcH=%9~9G_H0_9y>DX8Y=f{>8ALB1QcF9881n3dvsi0*?pUa+{2VS9SEfzHNGcI&3da)4b9z&!+gB0VZbuZ zr&Z)^2DC!B%)tqBd^oV7{ZZUwe#y=LwEnaY#Z70poF2Tn?(jvt$X>bV#x2{K+5TGb z;*-ney(O$gw;E@WhJDRtuSu7^L|y(%GQ$zud9ET+D>PGyM;JUU$rP4*)r9`!xh)F2)7pId#S3yiqUc53G2L6 z;mUl+JFh6&*2j52Om|m$M3&B0!w$%b3AU}nLxYx)vn?Y18LD>_dY59H+PEzf6IvO| zQcdruW-LWDgpfU{59fiA$nh{1sF|3onI5UW%M~CDLpDDJ86+n|JEfxeoL3Ywx(aSO49I zZj=tNJAl2J^q|?(4t5{kYTXguRl|3#dM=`*_Ogr}Lzf>C#IDr70_X-alia8F6#Al- zjc@LQ10HW?wmJp><)xjaJ#ifn+*fwTFpu>*lmNez?Xm6wVF%`}-^^2Q4^At0GOkm; zxi*Zg7DlX(EX>IxCTyWZ_h+CiIX$~5J@9S)`NwVzW1dtkZt;vOb2IM_wP< zK=1?Okn<6^L8cp$78?y#ihG~y7QW=qDFr#SJEc4arWBc2)RbcM343=+>35Z)l21~3 z4kyKe!uh>&$aE}n+@&txxT<@V53)a{?D{nhO({6;hUhe`V*_7|O7CkX*=;=x9)J9o zDFx-enAm{Aqq1R#EfxYMo@y69P|~7Z@_NQ9%>X|V#YHD z?0)v6TbTzY62#gOo57P5pGqsr4_N@j?wtch5+~e4BS}fW0}joU=){PNt_k=)0u(8bn1NSUObhk0IJ_%4Um_SN@u54ODh?Q=1l(-JFt^rz z&r;G^OfFVe)wr`kTdPTGxo*>#Wi{h1YE?fy2{&-^v(M?Eca8`g@FYYdTyD=PCB@Pn zy7By!xeoerG5x+11eZVba-I=^15RIy80w1=9!FQvxgZeHuVUkfFio+em{(H0fN`oX z?oHf1G|;;MdKWzN*FM ztM$}3UIWkN2{EC}n^mO=--sqWa}zjM6`Q`FZ=RCIQVD}JNR6FaqwmT>foz6?BG);3D1#B4SJ}o_bgQDnQm#Al_mG77HO-r zlr}}yXGW!wzc~0l9Cq-X`NhGP`9}xe9Q!X0zVu%leCPjr4!*Ds3#_d6VBL)Pd_z%d z=;}r4f=oe6A)5{_wJGIzw8RO8I+13fJ1^~)Ww@7dOaBeA={)4%Ge7L$JN^G5vC+#r z6>`-);p2eVW1ri+b@x1C{(tLTSV#QgU1<6*dKZ`udlz{BXWoS`fA%i?!fYgn{;Q(G z|6cC`edr-#Bgyri zkXzy1a7WyhfuO$fY8yNi-TZ*OKD=@iQ>R$T2I9x_*9>tzl0dOkXyn_MuWhZtxiOq zl01_*!OWnWhyF-$h4UyLxNXAF*S+V3#TuE22?AS{eOawrtLITRpY@~Nr4B_GI_4Ei@j)d2aD zLmURBaD2^&!ILHcg@bKZO!yum!SpnF4P891Wl&FsFGb+(n>vTKh zqLq+aqW9KN(GpLuQv6gB%7lzG*xlY!-=&8sfq5*M-M3YS<0}P#JL)FfYc)!Rp#=B@ zoCt)He20{CRbkFGV+IxEuzE4J{K(@z%qwd4)WxoqykFg@oED^aBHEp;88%ZBqWpcq}SvYMJ*O8Y~tya zgKvTJQF}z$n4Ny_PJ}o?93XMHw9~mE+u7&{pnNTe%5kj1#v&ix(f&%2T^+EOE{I1)31rdUxv6M_k*J6z-uX&xPNWIDY#q zO35I^kjwrgq0xpC6_9{-F)#x)`*G)p3H?(#l>TAQ#;_ju&Fd-)N|yj$WXgu#f>j2Y zflgn(Cvc#Ar!cW?bNzc8DY zol_H}Gbfwq-AQ-rKV1vKJS=jc%%g#*+{%fG{0cL5`iKM+wQG9whGu_04Xy2W2o2q#E`n9^5=drb_#QVmF-3pAPH*Igu zqbJ^?I&{$^E}CRjFk`<60#&Nf+3CRUHE2S+cO%g7zyk&Rrlw-^LV?%6`%MJ~oogB) zpx+eqn}U8*&~FO*P5;waE9f`<-|9E*J2<7>WRZr$%ySz_E<8GZqK}YDy7>M3_X4ba zM^}c_=VOgHxqWu&3k)6iez3Vx&DS0rfB;D`BzanRY(e#yc1IvDPxR!OMWSgS9J6=h zg+(G7R@6KLBZHS^pWga)XgTOTtvl#Fee!R;r@u!tGt1tOngBF2H~gs|`CBwo?vDYV zEd!<&x;-@Gqd|tzw4Iuqj9xDskKy;y)W_D#6ph?j^Rseccoy@HF(w-W zBaWg%)n(>B$x2@JI#y>px}kScL?3|^E$~X}oI{vm`>==%L*olE3wErrsDil(M9lwu z$14b-3I?s`C@-4qd3btK;&IMO$uUCv#esU8qsgE-bLHKX+OW_wIgbW z$9I0){GpJulkr1J0-xraJh24w0p88w_W8^-CvnayR}r5hLD%y(C)$A(!6M7ug?WH4 zue1=M{DhfurqV)DSeGO?$~T2&=sr7+*U8Fzof&ulQUgo4z!EO7gbOU;0!z5S60U!0 z2^VN&4FHX-|H|tCQXY`<94O%ejjaF6OSnMFgQ7en-zkq293m~5gA>3@DhG#EHj!Ve zdiUkXbX8K&dB)C!V@4omjJb z$_fqc)Am9Mh97?jN~nVUDwrIgJVppqrO2L__3u$0hru}E1t6?Dw;A>=o15yll_Kw| zG2pLy0KMest~!y6v#XbAIKpo6`1#4)8Mi0Tdl+fV(kNnfT;YfUYbt>}4bzT*o#%(L zdqUG|X0z2{UUV`0!U7der%qtwvyXShv+b0H`I?~o-$jb!OhXGe?t>wVq}N_^98^&v z!sb_DbBQyXu|hN+K-Yrb;Rys*uP<=!)oZY<@tA5D@2^ zL9p>2#Y~wr7N8C!#{iUteUX7Fof%dL>r~Ta{RomEuz3V*9s!$2z~+(vxZo5dLH}Dx z(C`5gl=@Ca&GZDeH^U`_C7(Anjpb!iGe$0ZWo%;GppAJ}=TD8|YC$N(LxX$2Fji*v zI@kUPl@Hiq%z#F>yS?sx!DD zEynsjzDMp$-^Aas9DoR|B$V2InED`kcv!KIGc6PU@S^u?y7)aNs5lRlEHo70L?(PB zvx!L4i|&cY+?cAv6VZzL{GB1HFP#CWZj=4AnV`PNDc zLCTZ3^2${|kdTTZ_B9;a%~GsBlEC01$v^X;9S_?Fd>{`3m$HQZDLaMGZRX`@z0zu&v7i3Bfb(i{vbZri-_{~(>HJf!VM{* zpTsAhXbEV42MR}o0VAvJ20{bA|02t|M1iMKFz8^Zi64@OE1K%qG2PVnF$aBkh z$)@9x5}P=Nizyz*7(Ee+QcLQU{jK8xC?y3S_x{*xNexWY_W(*wXy>-2rckO1(4CU+ z@z+w5gCOYR0T9&q=Teh@5%+Onb)VX(+wGSB9Mzwtf`9^GKF8zCR$_N_G)tr9uG`&?+LY#Q3T zR}%+>tAUtw7xn%$WuuP!r%L*Y_XkN(*xw{U@r&8%_t4?N*;!A&M5F(%v8}5B6MlCR zm~O~-NzfAll+7i^+bkx<H+KRf6h z8dIooDwID~XL7V?3eKJVIj8JKafa4#zL|Qsc|Y6oJ&1<{NfHu~5(J&BUF{Y2SCX)< z?jaeWsCgohJ=4uD1kz|N_EsB$afOC)qNANT2h%F?4pJ`l3}t^&)1!p_)(*k4nNb4O z9}w+h^tTedZ-lkQ^t#ibpiA|PxQ)!O1jw0QeQJ(GY-#*Qzy}B?#RU0U;uwZR-^LN# z;QdR8Xl_=JyRuOK?ym&MR20!QQZM*35j4OO!c57r!>@*H5dSmT>9wY2mOfcoZ=Ov< z$s%8EEI=E0)u%bxct1A;@S?X5eGyT}_cM#kRA3?q?0o|X5ZL<$_P&9=Z(#2m*!u?d zzJa}Ol&4q!@0JmRy>DReo4}bOi(qM$x(_iAOahX;-dI_>^$E4o`0rHP6YdhATpUkSA}AL(GxsPzbuPLl947sd zE-yzG)H98f7|$moW+E{_rubF(4a&T;wmXPhQ>=wxx2c_ z>|U2&V&W4LhfxWzFhm_YChV048|HfSFwEsD-sph?c5HkA%aXp8x%Xv@q{Jm zLpKgJq+Dgzm#myRU#MKTy&2KPhk}!FNg^yfB;Np0N@+IIF+Kyr#j|y*- z_xO6G^fSeWsXgR_28>7<$~9jFQTu8g{|-X? zM}J)W#uD_`+2&vt=;J{Gl>0XcP}Ya+nPh+fRro#4&HF6@N)Nz-x0M6K&A}|tMW~X{ z4QO4G5cByqLaED~3T&Jcbo?5{Xk-`EAQP0^t)*zTv*Negqa?XH4~!(tD)Lp9LFowr zFM7UiFCCh0t}u&9RO)Q3xp`L~9(zzv%yvWpS)kkW2Z$1o1rin9gHk9AB!U$Wskw}g zfVIDXVEXuka>tfpl3843xM=kS+3L%(9#R7BE4_8&-1!VQk{!a==1aZRu7%s@Egf+V zw!hUV!~l7o@(|@51Fl1Kh?iuHW%Yh%F>IvlFJFM1?P#lDaTX{1(HB>xsIZCw#DF%g zdoH|-qEC5iL5Wny!b+rHb-w)!U-q*{EvdH3pXaZx^Jnb9=b5Rf91g zFb4E54bWf=2#f)NF(5Do1jc|A%tV@Z_a2kRPu>(s+-SpuU9gHQY_*yP_ZxR~A4P<8SQup0zrK`5aIMF%2YA~*X2zvk&`w3OoK$XxU)6Uw;H;cO!00Z7bj0ypG4YPFTxu+vT;EXqIwyS z3gO5+2P^YAfrd)U_>@*JqxgZ-qCO7S<;TCMG<0vg-qc%eFa^8j7=EV!nn`ji+Y(Srhw;vu%Uxv{% z#^*8eluo$`6-ti)O4uee(iGS3p4!`73>3U|bzv2H?{k=eEH~rE{kpb^wr;H8!2PE> z%G{8yKVF2t2kYe{`06tR7OEER`+DOneIA!C(Lm}{r3fYAVq!ULnAQNHB}vY$JrdL@ z{17f>%)*fFg1Z)=v%g8`@^TD)@sBK$OXHfCIjUhFsIa?SmNx$vx#S=UsymPd`g^r- zb{pi4-S3&3gm%`dNnK*lDSwj&difR$9@D#zf^zqoQ@WtzNret9Ryd+vF$)@w_dtsnc&uEHt4w4|i7n}3eR8_k<^{aT>LEr=UQ8!puE$1=@S%n?P z56UY*S;-b@>A?C9tFX~QMnuFpL;;{6JwS~ptn=4IC&vV&h++T>aqdbH8-1I9mLh~b zJ`r64Q4p8~0<%D176{A&fmxt`d=}`(h9EM`p;Z6YV%-jVUzH(jFRTlK7cMxFtV!G7 z#lAE`+XpjpD5-Nf{=NAv-BFowWmV<7TLzIoJp#4b|3h2wf>+W0q;*cb4y#<|yTwN3 zv(gaOe++{D?uMXG^L=^c00?UA&sc5vEeQIG#DXV3#Qq)y`rddOU)2!yB96tFLMP`P zbGk>N7rcpplMKBcmm_#c@)I@k#@x^;J&6jtL-R|ZZ}riN@sy9TCyida92|L2g|~&- z+ncb8<)c4)Z9{$Q6`WnY6D`FgD?K#*erAM9b9_Y)ZU+*}G%0PpLsi z>l}%z&B|Y%nSOKUqW8>Yy1&%HopYp#Lb?><{vb-*e!lbo1_BC#2|O9#enYJ}Sj=jZ zu=7p27kej2#(C~GwI&avb0G#x(F_ONN|I#b(WS9=M1><>^6EWG)O?u?qs@UfKyX9h zIAL&{H(@z$-nSQKH(;OKPe@};d-8jtGt0>^%T?JjUrRI=7i8rIw*!IOfxzuR;C3Kz zI}o@X2;2_TAFaQOds^#lqr{>N)}2VYaWaufI$V{?G&slP8nrvYl3^zkW-ujQDQ4Zti=-d*b@XCXKD zXA|m;w_4}Y2UT|ldAHJ%cfLXUR4X343zfO)IIRCaodv2SkdlH+X-*{{X}36&3}bmJ zgX0p9c^-+p!bSLsI<}eU zB`^NHUV9((WE%zgVtG_DhlhhwY$pj*9)ZW@dc{2uyA`bM(|E!l6YD{%Qp z)V8--8CVo#V2<<0JW%xk2$cP|@<1MmC-1m=DJI!%ZQt7x`aJ~Ne1r`j%K?-UR37L% z1X?_SM~X{s55={v&AkbcH-Vc@G6^sj!;X~UxnlpIK#r#kj2aIj9ZvZwlb|*60d51e zqA`ktnn;a8v4{88tDG;I% z8$LS;%r=yJ6>PhOa81UKiDA+KDnsWdeeqXS#jyYl%I$b6`^^*cwM|J)h1#%DU zT;k&X6nmuNR|>R^;~u@X3d2=&sZncz7;(`xfkwDT z6MXVKl(6b)>Ao<$oTD&aJsYT?iSzLkB)NM|PvFQcgzBn#30`sOtz!1)lgA$zVRf4K z5su%;*#Qgha~jvJ@8^{WT{vTNp7%a?Q!{ z$0DJ5Zstu$3j}gb1aU<~(eme7ptI^PG_L}MyjQ4NZvVNEH-bAJy$EJgBk^xK;i`MN zAA+55U>pdH1GRvia9}4K*a-)A!hxM|U?&{d2?ut(kx4kW$X2`8o59YIAV2ZvR{<#Q{ec>IV3Jz0qw>LVB^FCTR_Fu5#D@9uqTjrZQ2 zWBZGX>n@P<-b?@S$0=KS3#0E)hGE^`E$PjVIFMTp*7}di`&Ki0O7i{tUWhN}PL*^%5&k zPQKX!JMb<7^&2+6mrrAc7+NRv_}pKv50;G@LmsVLsG>o%$ zk!$~S+E(Ey7w@v5R!PKY$_vTw%Yjy0=$`YO-!}j1Ve)UZh(wA_9TRXl5Euvo13_RQ z2n+=M$2Obn|E|qORjjtt+23v<&eK;d3P-j~Rl}P!{93qJ_wh4Y$B!&?J&SN_VZN8D z8mt%{7oM=rOBJrncf9k8l5KsQ_rr8|rAK7xY&GnPLY4`(t;0iumXWh9BK;YvcNBV; zVw~EzEfW)38Ou^l@2F-hMKy$wJ*f}pfsx4ZFkX@UTDPJgt)vx3`>Z}eU{tE-A0G&s zIItWj^!+pw5UVcs)w=@Cy%8Q5it+Lj%+@|FGRTiOuhwmqv~;ZQ%e}pL4{suwfTr0%@zyHMvo7wbynCGaY?-zd#Zze`Y>6Qbzpbkx2!-7@Erv8!c z3W_s}>_y{$UHH>ERJ_SmX^=;FkOy@u#6%&7oMNMq?(IKU0Vyg}U1L^Jws&4yBzfVl zd=Ll4L4aIPue&FEh{*;U?@`Rv)?GW!fRLNU?taGe&KGC@T>G~2ZRwehu6J5i)}OQ6 zUE(%DB2aKg5V#`<+z|xwAdm-vJP71LAP)k05Xgf-9t83rkOwhP@P(gvL?_AY@Q>s{ zaNtw>?v9|{dbr%Y>m=iFX*gLW?=@N(*ohM-U+6P=z@p#LSE_w)hZ8Edd_;;46q9%Y zb#P`4;R~vn3j3RYMy%+c?)BOWlfeldyCt38(=>n{x!hi?o(Cx1f(G}#M21_^+>fW+ zy0PwhquSwK?@1^K1^`Xv`oAewXMpi}TMMfwm@#4B$n5H)#QpXNDY1!TxR@fzgjy3c zv4IhtUO|XLOF-l(_QN(pd|yYIc)9^q-z$d$qDfg;MSo#I2egvJgXM7lCKnW{rU5w% zf#5RYVnR^ANGPyF)s*1a@3KZ4=kGy*ZOYBD1u@XyZBve_!GYJV0%FO*E`8Ypa* z3BTi3;k;i)%Hzwzc>DMT!8R3$UtzvyVC8O$%Gf|e^w_cZibJAEkZ(5Q`nrQiYDqC& z#(|sJoW(4HnP*1*8~f~sXF{hy z`z&am1?{t-eU=N9OF+25E$Ar$y#s=d4d^OCA#TP4*j|aauHWSnphrc1D3mMUbYKvWKqd^cd|6Se z?T+1*s+6%?qf+@8X`S%7A5cZMVfyLeg+lauT`B0e#b5X(?O&qmXW<%%_VrCf0cyju zloVR0xYGJx%b&bj`J5>XofJ;f)9*dFuc#nRymfU9rtR+d?IB@?+-0#*B zaRMJ>AlvIr{kk3f4v(j&lJQ9#SyD<9V>gS5i)aeQqZxdw4PRk>eJ)?Wh)fA_+Iph4iKc(Fz_mNLBH6wd>q}+n zk3e^8!>6eu@7Nr0jXs~*DJ@2<1XfzfV9@+p*HmHKQ zC#X^TKPMPQoV>94RaoNG#wP4u)~~Kv;R73$wd&w1 z5EmZOV7QD71~pj?{;4cyILur8d1w~Xyk!f^YF!_i?SN>BBiYOw@^e_7T!ykmaW)cX z-X1$hdamx0o=@L$)=yS$*wX){GT-zzRW#Dn?Pp~^P_D9TrZsjf4^o~#J$IM^?XFb5 z<2i{x-l=(60DnHi$ET5~)Nots-jlDAt!wwqHt$xG;se=gE*xYqTe4lGXr%TvJe6tFx6EKdQ;Q~vSg zDaeE6DSXh^@^>;6Phfi^A)Qyy7>&)CXcUg;Jg~R!yC+#`?M`=na(^t!Ph=k@?o`@0 z%1oKjcj4~OPbZ7qj1zzK(XDi01^);j6$<(NWl7M1Fpy;3K?s!lZ;MlY9|r0l+T1D! zAkf_7<;uq2LZF;^T=+l=-uE!jcL;P>Qv~wSbCyGx?5MVvjLt3Z^*Zc}0ysE|t>@35 zdr!GlSBhpkD}MV^RMxo2L8cQCW`FT=N$}dU?pNbthlU$IlG#LLiU(~Xa>MZM&(+Dy zCc94>|%{KP;_={IqrJZGD=DIql2_;@^WN3vwaN$~HUR5*n$ zErVXc%vmW3WPuh3>TQx;Igc((AqC!UH#G#bzj1AN4HvP;8h~KzcC{$R1o>Lx34SvAP(>dZyy9ttw-$v+7tbp1k$rOR6eD=b=CvXi5a}QLFisQs7;*?VI-V zYhKSn$fA7VrgD}t5m$p4!(P;K@K0VN26KoA6iAP@wBAP59OAP53M z5D0=m5CnoC5CnlB=pPP(c7eohc?z(~SX1%Y2jK^qUB#Y}3~Hh5oEXG6u)SSiFALz* zcr}4TsM6QYkjF~!{=F}matXSDJI?I~e5ZNp9US(JsPpP#p!epM5Rrm2tkR;z&czE@dt zT?puueF2l3qUu{%d$m{S0`$v{$I@F+qXZO6GPvr>rW8F&X))H2x;ct;|J65pFc9?a z0129-^otRu^o+srO)Hl1S%#8C@VD|sppgQdDOe;J*!Nlg+MjZtb+Uf-n0pB<^Ix3+|2F~K+x z7zYC5KwumQj01sjpnrNCX!sxt64Vn8YK<}D^NNoQI+OGDH9oLo+4xrc3Ay4^v(<^n zQ<5vr7xv@Ty3u4eLUb7ScNLix5cQGn!=Np}4FvyQjXuy@SJ;a+;Ds8YfHt~@v$&Ff zm~12)55k}ae}qB*da{`**?ijrOg4$(Zx)7rJK1d4;K3*Q_F+&i4ys1~|Fn13e@!oZ z8y_%ogTY2BsK@}xfr!*01W`&UNs(@lM!KY>8XCS`n?R!=K z6@2Dlj0S5M(h<~8iv8L1is-LHK^5Z7C0?x$68(d}TIziz-7VU%tjm}~>A~+=mp#7? zesdh_f@589tP75H!Lcql*5zN%x{UtR)BNc==)ya9#m+&t=uV+=9Onggtnrq}F6v;5 z-MzP;OCsS%ND&XNH@5$Qy)f+ssD4LVEX6kn?0=-5#d3)Ew+096ieYm5ZM13X`Z?M( z{?BOh%Rmr?D*`VXW?^Apx09-qjXGIEFqQrP_-@M_h_VEOO*W^`IyaDhwYUuU3z*_* zk2s zJ6``FxzbRquH_SH^yJbZuzQ;8S1<Q%=UjulAt6O(EhDUauaO zH2mgt!M@nohD6>zWqYwN4(89J&wT&Y{oH5219+QyehxMPzYR9OvH2gNcGj_-O;h!9 z_0fNKHmB2vAtxZp%5w1Ee!9Oq8>DL+rwj>)+A^PGh|(N@1Kz9dK-8URwNdi$kqKIP zhp1-4%Y7qR%MAAOL8?0b*N3mo8wjUbz-y z?8M=gqvRrMjd1iox)wOxh##8ATC@naQW3jsgG9|8cnC7RRU#f$$#@2$Y>9%kR^dqJ=$vHLyaM2l89j-qub$$Q+ELcX9MCj^4%5yEu9mNAKe3T^zlO zqjz!iE{@*C(YrW$_g_x$o=zXuk@{mbE~+9_Z4i2$sP<8$AT!Yow#TV=gFEvyjjC2$ z9p;i|DU=8i4Cn8qe}{0F8P|H=PDFi&V|UXYqX^oiPOAzc!s}(MKH2L(zS@`vx$_M+ z0h5J=@c3jm{P@V{8N`L1c}qQj?iAkTCvC=p#Dh{J;9x0ciK09Zz6rrWK-L$cG$k;B~&FLx#ja)z3+D*-G(LNz3&I30v{jWY{u{Rq75H4`rjZf zzl_8$ELNWls z+Wcc{z?7>E*tVnu04P4}PxtD>{ARK55A>nbL7$sBIO!w}#9kuGhwTZFmZJ2%M7gp^ ztW5vAS3it{G=K+L9O~J_0N4d%D=4nv-FH}ZtDCv9K24svqtVrrgoDnb1>bUOMu*w=b2C7GvUozy{5>g|AwI3yEM?{S_9?Gt_R|tpE zIZ~I+!Kd0hz692UK^$qa=GgGUllPz%2ddK0_br@~l;!uZumWi>8WHAKeqYp>;U zU6S%9`-iwEpUZm674IfqW$4)5q)Ur?8fnT6!@!rEm{aNu7Ip0wG&dLYCAzW4m$#Gl z+wkcHoyP8n!&og0hrPd=rCAvt*`3&jG&$*sjP<#R3eVpu2$DosJyqw3y!Ma6By2Vi+i!weE$S}Q&5G~SjRkL3$ znR-u%UM-S%e~aMEiw(gWfoJg3Zd4{D+XQP`Cu!P5jXeDFN?*(-@WJ4XjiOs0Z#_X0 zf6NiD)bO%?LgAu2y#_|B+!P%N_Z@O>7*K2&ve}B8TFsbRv+x-}>=zh(URku?FPZ)< z>$hBd{7wi+NiJ~>{E&n>l0@_=BTWG?E9Z=0fh^PNAOrdG!y8)+j|a$mm!oNSrMNtE z$i3|b-c0@d2Va;$1l@8es;eOCjb&8da#RJsKO$?oSavgs$ICMADpdr6{~jZ-;`=ti{5;nG4VvK&dts7{H8*~lHIv2OM4Xmx!u@M zap`dVO&a|b4gC#4{WVv8WrnGld-a2Lo(XO{=BQ1cMAVC^;c>p^D^n&!GLZ#fn0Jc^ zuR}PO-aJwDyMvmy7Jtlu&V8a$Mu%9e64cKS>wkjTOKe&Bp#%^oyKyl-b6JNIh{K_> z!^is__D6lFqY6nho!@TR@pYf|{^Lo5!_cOEWj~OVlzJ9W2|z$S;}6Wg0C|-COoDth zoAhHgXn%$gjAW%R?0?8nL+I=PNuHr9K?;l(^vj4&gesB9P|w8%<@4n?l61c$TYE`5 z{t|Ss#YpikWtLvQm7_k`*&ULyMOFGvYnC=3EP9U$h9r>{zKA$^v|SL^Rq$xFApB!N zgy-rL%P`(S`g^$?uVtNGVrW;#Eu%6AX#-QEzwpCa=~buO;S%j0dTnNU?XDATJfD3g z-Y&U^AwJO?%yTrh3w)NTZ;Nq{%KDV-A9Q<G5wkc2HLUG8|FffoaP>yre>OEt|O-RN6e0jhu7~?DC`qP#4&KZ*i0!f!IQIg0Do9pfKj5wU@8{SM$>-Y9s zO24dnop$5iVh+A~xRZ=7002ZPtF^_u$m{$z)aiY3mph*O8?d)EfW zNEiZ98ZI3}gnA;zd)oG4Pw6@%SV$dGPQnYDZYdpIOp~YgI13FObEtBXQ96=y@`Lkj z#tzBxL67F7ovzTInP%+>RwNrqPrN!BhROYl0P3gOM17H{j#qwm;m=4-j`u{|RE~%- zL(W@wyaO)_f9bMJO3S%m8W%*cubWvwB6e9@UWF@aMrxlpery@vfSt3|ioyNH;yW2% z&rZR1m^Y-={v4$7C?!zG zkS({3jeyWAis8AGU0IxeOq(s&cKg1$Pyi5-nstFA`q&1!e8)|;{EBYnV7 zadr_#c%XaNMjBg@InRX_EH(`~F+e~uDP`zVq0+@;~t=@6(MK)DlqOH1~ z1L9xK&D4x^a&K89R0>;xk^qv|_>_3`v9~Yc1HhL` Q3Hk6I?h+6vtI5#+2fmTVNdN!< literal 0 HcmV?d00001 diff --git a/extensions-core/orc-extensions/example/orc-file-11-format-hadoop-job.json b/extensions-core/orc-extensions/example/orc-file-11-format-hadoop-job.json new file mode 100644 index 000000000000..43dfdce039e1 --- /dev/null +++ b/extensions-core/orc-extensions/example/orc-file-11-format-hadoop-job.json @@ -0,0 +1,85 @@ +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "example/orc-file-11-format.orc" + }, + "metadataUpdateSpec": { + "type": "postgresql", + "connectURI": "jdbc:postgresql://localhost/druid", + "user" : "druid", + "password" : "asdf", + "segmentTable": "druid_segments" + }, + "segmentOutputPath": "/tmp/segments" + }, + "dataSchema": { + "dataSource": "test", + "parser": { + "type": "orc", + "parseSpec": { + "format": "orc", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "struct_list_struct_int", + "expr": "$.middle.list[1].int1" + }, + { + "type": "path", + "name": "struct_list_struct_intlist", + "expr": "$.middle.list[*].int1" + }, + { + "type": "path", + "name": "list_struct_string", + "expr": "$.list[0].string1" + }, + { + "type": "path", + "name": "map_struct_int", + "expr": "$.map.chani.int1" + } + ] + }, + "timestampSpec": { + "column": "ts", + "format": "millis" + }, + "dimensionsSpec": { + "dimensions": [ + ], + "dimensionExclusions": [], + "spatialDimensions": [] + } + } + }, + "metricsSpec": [], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "DAY", + "queryGranularity": "NONE", + "intervals": ["2015-01-01/2017-01-01"] + } + }, + "tuningConfig": { + "type": "hadoop", + "workingPath": "tmp/working_path", + "partitionsSpec": { + "targetPartitionSize": 5000000 + }, + "jobProperties" : { + "mapreduce.map.java.opts": "-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps", + "mapreduce.reduce.java.opts": "-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps", + "mapred.child.java.opts": "-server -XX:+PrintGCDetails -XX:+PrintGCTimeStamps" + }, + "leaveIntermediate": true + } + } +} diff --git a/extensions-core/orc-extensions/example/orc-file-11-format.orc b/extensions-core/orc-extensions/example/orc-file-11-format.orc new file mode 100644 index 0000000000000000000000000000000000000000..41653c840354a2711a262766ad0f170b34d2062c GIT binary patch literal 373336 zcmeI*Z-^ZCeaG?H*}c2DIq9_8JDo-+RFn%r!4HH^|4tHVEC?a=i31^iZ1!}jZmD!C zk?w)%gByZzTkr!%4~j8K6R2Bgp|wlX5(J5lxG{cEYA9)1T5t<}kdg*$Ts^5sPMtq{ z-jl77y++Z>nb+;@&ToJF+qs#ye&6@!w z=w^S|>}D}6z4YIkaa^8DabAP7SeT5P%THhIgsZ*nlb$S&tQePLcoov7R*&_?>c3tc#WPE%&Ma*_S>7%_dFQKXV`a2A2Y>(q2q1s}0tg_000IagfB*srAbyx(ppLbgL$A1cz_3ZfCqSh2Y7%7cz_3ZfCqSh z2Y7%7cz_3ZfCqSh2Y7%7cz_3ZfCqSh2Y7%7cz_3ZfCqSh2Y7%7cz_3ZfCqSh2Y7%7 zcz_3ZfCqSh2Y7%7cz_3ZfCqSh2Y7%7cz_3ZfCqSh2Y7%7cwkf>$f_?mx|u%)!R%(S z@x0Fg5Az`>pgziZpdQqNdf)>- z%n8VYavrD$^`IX3fDidFCm;{Xd7vKDgL>ctKIFrkfIKMYfqGC6>VXgVkPmYL@}Qgt z>OnoI2R`6KKFkTogK{3I2lb#H_<#@jFee}n%6Xt3)Ps8913u)#oPazi=Ye`q59)yr z_>d2C0`j1o2kJpRs0TjaLq5z2$b)hos0a0+9{7L{`7kFS56XF<9@K++-~&G7!<>LT zDCdECP!H;X5BQJ|a{}_9oCoSbJ*Wph;6pyl3CM$T9;gTPpdR>u5BV@BAP>rUpdQqN zdf)>-%n8VYavrD$^`IX3fDidF zCm;{Xd7vKDgL>ctKIFrkfIKMYfqGC6>Y?yKBW$(;4;YvS%76-(kO`TP37L=ynUD#Y zkO`TP37L=ynUD#YkO`TP37L=ynUD#YkO`TP37L=ynUD#YkO`TP37L=ynUD#YkO`Ua zp_wop3r2f$009(4! z>_3@zt#@fP4V$CAIRFF@KmY**5I_I{1Q0*~0R#|0009ILxLsbPOUI-1nU|k>^~GH; z&1mlAkYZ9XAyQMOa&Dx`=-xVEnQFz7a8&}4a8)Xk@XoNl+s7naVcUxCdmFoEy(CP+ z!+r=NVGppDe!n zYQu*bfP^b-ThaaK8*=#N+mKV{r5O??VG<_c>I*C%nmZqD013MsF?^o69N}`r@ar@y z;R@SUbPrGXqdET*wO=b?D`6{PD`6|)5z3u}NtlF5n1o6A5&W>?a)iqf!+)rCIijMJ zE4qg#JVIFxe@*kF{T-c^u$8ctu$8ct@CfBj!X!+>Buv62{0M$naXG@}h~YmhyBtwb z$`#$i6CR-~U5=<)v3|76F2nO9VGRn1qK{1PNCtqN00v!X#WHlq5{T!z+S>D-==DJv?C&t`SNSCgI@~LBbV^sOTP^ zFbUTPB?*)8@QNVe3Pn_O4^NncYlM=7t0er_5AL7UK9q|3Wf(MqJFP8`2q1s}0tg_0 z00IagfB*srAbeZFi?)BbU8ib?0IRFF@KmY** z5I_I{1Q0*~0R#|0009ILxLsbPOS3C`BfU8Q1Q0*~0R#|0009ILKmY**5I_I{1Q57g zUZmR^o4ByrUF%)xuCJ{0meV*K?acuofB*srAbT9kH*_U zGEr=Pxjvz_xCrIOcnjR`pQ=8i!v44BS{{D#l*@*ndB+2X9xsFF;S!=!M zIgtFtH^QRlK(gKmqu@jDtsLS9Mt!h1?-?t`-Tm5K&*HI8GkSd{ZpY0a2s)El`>|FW z{?}wY8;4oPbk>pehV3^R$shmSo_Mb4ZRfpHQQB^&hm-XAe{RMHivF3re>zIjsGZ%D zG<_xeWBgR{(!u_2ji7Tl8!>-yGCPdy>z&8Cu?#RMaxr^4d-lf&}eeZtzSbVT}_rB=uW1V*N>WjPD=`$}smHg2^{_Mt| zOuWbQ?PJA}bXPiIZ*AvJw%`5w%tAXJyRh0_>vblsbk|o_ddu5KynTEb%l!D_{P^zs zzNxsC<17D=zgU9#drrQr7Cczh`NF=ESix`Pb6fvnP)( zE}b~Nc>2V~lN(PiK6&S>X=7!yHwSyyv_@BvT) z96*A#_CT#qUJ8go>i`QO60{Pu60{PmwFhc_@>YWB;hT@r4A=;=zg_>;Jsc``KluCk zjo{a_eQ3vd1ws0F_7~;nZE-LU)Bz9h01xm05AXmF@Bk0+01xm05AXmF@Bk0+01xm0 z5AXmF@Bk0+01xm05AXmF@Bk0+z^{h~va<+|Zsw0cFuPf7JnwVB!+gjIsE=|Us0a0+ z9{7L{`7kFS56XF<9@K++-~&G7!<>LTDCdECP!H;X5BQJ|a{}_9oCoSbJ*Wph;6pyl z3CM$T9;gTPpdR>u5BV@BAP>rUpdQqNdf)>-%n8VYavrD$^`IX3fDidFCm;{Xd7vKDgL>ctKIFrkfIKMYfqGC6>ftUQ zG{R;p@PL7NpbV&h37L=ynUD#YkO`TP37L=ynUD#YkO`TP37L=yKeh?ev0$_}2Y>(q z2q1s}0tg^*^T900(nc`an*%@q0R#|0009ILxcOifWLwQ}`RR+Ry^SFJc;APDFb!Ly zy*U5`5I_I{1Q0*~0R#|0009ILKmY**5V*DTB3&9hnCpihdoKeUx_y)>h_lS7J0#e_&rnaa75Dx-Vrgk`D~OTtwNM8Z|6Ou{?E`feYSaD{Cvy6Bpwt>_+}@X-2y^nZkvu$8ct z@O`rQ?yC(SY5)?hux&;6qi@LJmv2K(nU`irn1o4~gsU&Gd}!``v;idSa>Vd?=5mC~ z5yP+3tb{9UThTo{;g9D0Pt<;`gsp_Fgsp_FghwcM5+-31CSejL;YaYpipvo$M-2a= z_8TLt<>4>64gFV0_x=&T471Myt8iY-WFeZFi?)BbU8ib?0IRFF@KmY**5I_Kd zn-69|=Km6e00IagfB*srATV+QH?vD}58$>nHgRFKyVkqXU0+%0EvIof+M5GF009IL zKmY**5V-kZ7Gxt|usQ$)5I_I{1Q0*~fnU)jyl>xn{rk;t2Kn7&Mff4G~Vn7CDKnARv_Xg*CbI{a!@c%Y? zBM3g;_n{z2pZnok&)+!n^>X7G5;xA61!jR+z>0Y=Z~+&X1zf-d*2z92NJD8T4W*$p zl!nq!8cIWHC=I2dG?a$YP#Q`@X($b)p){0+(oh;oLun`trJ*#GhSE?PN<(QV4W*$p zl!nq!8cIWHC=I2dG?a$YP#Q`@X($b)p){0+ziw%`5lnx)??b_KJRXg=gJh!EHj+-U z4U@TI+f0&T+e!`=+p*-+#dbV7Qfw#s+bG#pJfG~pFYfPeDw!(!clDp|PG*be)BXMK z>A&9YzrHs)RJ`8l?`Nj}`fR^{Uw_Hf&+?2z+)?b+Vf-sIiiDTa2Ee>ztT?Ih#HPCLn- zV(85F(An)DJKNvcY{of1uet3J`?p8z?~mBuACV;QcC($1wUg`!^V2(!{Ok`JMbClc zbDb~>KJ?zo-W7ge)CYU>p0Q%w-LKvCEMxYAx8~;eWZS=OCBJ-j&*A@RefHTS^LwN3 zJin*?H?8EEA59$oY3sAE&n*02w5v1S2%D|3@rh_M-ZPUNn2(mPUcG#=yS6w#w*2(< zi;MH)*H*87?fT+;tGAXtj$OXGb}8%p?aNoY+0%)YtLw{`^HKTWYgxy&>tE{?Bd+vT zRxV$hZ(Z(PyFTB%(*51}@cOm+XyxLC-j(j<#f1z^c5h_&FuOOidn>z-W%u#y9%c85 zg{kZS8&l;z7RI8u)7Z7JFY60;Pxp^!`{!lD^Ic4oquS-Dy`8F2mww10%jIjk^Ha}%YFzx(`wjigZ0PYbaeMLj ziIb;JFP%Ah`qas#tkrpc&(eJ7?D13C_nEV27mppyS|^rHoqzs`=npg9E`-6<*KakO KU;fG$+W!y8RG5AM literal 0 HcmV?d00001 diff --git a/extensions-core/orc-extensions/example/orc_split_elim.orc b/extensions-core/orc-extensions/example/orc_split_elim.orc new file mode 100644 index 0000000000000000000000000000000000000000..cd145d343104983b4b09603c251ee749e5f82cc7 GIT binary patch literal 246402 zcmeI)Piz%Qeh2XCKhJI#W2WfAlWOyiO-X|e%1A= zyIz00-{)XU?*~8pejIgGbMIU{8Osp5Qhsl|IQQ^D|N7?R6JdBQtu6E;3)|1u;zF$QSl8u=tMjqA*j1NS zR>yr4kHxy5K74{{sKd_QZgKQxk|$5|;X8Z%L5FOUJh!Ood`4E=YhN8-0 zcn~}fz7Kr@o&(Q;=fL-X103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3Gc)uJ-`!`%G0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^yu$>hp$t2F;qYw| z!Zhr?c=ROZx%Z#Hl{z^%&k@$&4k6AoGi^w-LoL$jp_b{)P^)xqsC8N#>L|_UTC_#F zI@B^<$hB6q{(m8dJ^{~x=fHE|d%yt>aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<701MjZ`7xn?rNN`{L1)nCISq{#NSb;5;RY)6BFX%?`Clr-xdmGefP?xuMo+aj2s-pKH+;>FQ9+bRpNWEmNIqm1`55{C<_D zkIOildv&_kpJJ3d{nVlzr5nTkD4p-hCgiQ`>i*DIdwC>|x~jLMo*3;8T70<>ZbW8cY@#5V6EnZqXanM;9I`^Kf z#f4bqe%IxRtMjqA*j1NTE|2>r9*cE9ZTJM!a4p}N-QwuYBu}2^!*}-jgAUmyd2UhF z`4WZ;=$G~P`}g8BR$DBT$LN9lZ5 zHX(0iSNDg$+RGzx)KyJB@Kh{A=swKdE;e5m?QW62`s#XI9EMNj;pH|~pWXbt8{OXA z*ji6dCsjNfi_k6Q_r{BJzkb@kUVeOH|7I_(Ej)bi;KBE{pRL7(SmnX4%M(}UV{x&o zuB==c_f0$&>wf<55vJkmBHx?cKcJc9$nVy z!*k$!zyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N@tF?!b-x0JIX2fCMBU0SQPz0uqpb1SB8<2}nQ!5|F?ogAF!2RhW_o7m*{t2BLF#@XDf)4l!_qulAI7VRkA81_f$d{;IhZ)I2ahrZg&Bk%Cu zIJE>MAOQ(TKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAOQ(T zKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAc1$4Kpb^dlYevN zWX!*!(jCv;Zmx;#ZjtWXy&e~b(NlSJxsBE4`mJtsdvjxJJw2UN@oX$Yx0K%-FU~!D z(7(R<_{9F_Tv}V`M;5l9t;K~{<-xAY6IbVBaj~mDTKQ<)H}P1k`}xBsn1=pGZFY;J zHkm3)o8-AgRp(0>F5uwv_RISzcs~X2Iq*H;00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aNvD+;K%y`XeA&42}nQ!5|Drd zBp?9^NI(J-kbndvAc1$7z%-O$XD=MSO+uK4|9J7}Nz8NaKYuHAa&Vp_tiK&XoM~p- zkYD*B3v^dmJn$NXpi*$9UWx9}S*_Nr!waT@LO@6;h)5m3;&AmF^ z>rXMtoqlT3j?#@`f0WL5WfSsNc6EQ~tGzrDKaW+D4?LcmZmy}@-6Flbb3LBQ`^#;t zesuF@H@dyKv9+F_PO7++hsTR^4 zXfZGt8yT2fHZ@>bB*4t9ZE(rhz@USPft`WDk3j-RCM6aDNhgK_5K@wXAtkZI5QvR{ z*q9aQ5*AJdhO=jS?E4rPVoz=4YB1nwd1&IBw6gi=%V|nGe*asaZgb?|!yiAKtF}lq zZM7GR2{Ft(x}dGu>fDwm@v_BSZT%s~ff~(OZnUizD3MKZ%Q%uyEnGZM3G`7FReMn-Gfg1xeLlh&!;{>rA z$5oD=J*cyxC-6}8LGR{3&#BGs9jy~PXNGodRxvi;x_)Pfef%?6KLqz=EE%Z3>+I61sEk76qJ}4cr>yMn47*ZGy4ZQ Givs}V|I>N^ literal 0 HcmV?d00001 diff --git a/extensions-contrib/orc-extensions/example/hadoop_orc_job.json b/extensions-core/orc-extensions/example/test_1_hadoop_job.json similarity index 81% rename from extensions-contrib/orc-extensions/example/hadoop_orc_job.json rename to extensions-core/orc-extensions/example/test_1_hadoop_job.json index c85ebb6721ac..df65ffc73bed 100755 --- a/extensions-contrib/orc-extensions/example/hadoop_orc_job.json +++ b/extensions-core/orc-extensions/example/test_1_hadoop_job.json @@ -5,8 +5,8 @@ "type": "hadoop", "inputSpec": { "type": "static", - "inputFormat": "org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat", - "paths": "wikipedia.gz.orc" + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "example/test_1.orc" }, "metadataUpdateSpec": { "type": "postgresql", @@ -18,26 +18,26 @@ "segmentOutputPath": "/tmp/segments" }, "dataSchema": { - "dataSource": "wikipedia", + "dataSource": "test", "parser": { "type": "orc", "parseSpec": { - "format": "timeAndDims", + "format": "orc", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [] + }, "timestampSpec": { "column": "timestamp", "format": "auto" }, "dimensionsSpec": { "dimensions": [ - "timestamp", - "col1", - "col2" ], "dimensionExclusions": [], "spatialDimensions": [] } - }, - "typeString": "struct,val1:float>" + } }, "metricsSpec": [], "granularitySpec": { diff --git a/extensions-core/orc-extensions/example/test_2.orc b/extensions-core/orc-extensions/example/test_2.orc new file mode 100644 index 0000000000000000000000000000000000000000..7e648916eaa4e90c336cb1dd9313f028d94c37dd GIT binary patch literal 983 zcmeYdau#G@;9?VE;b074&<8R_xtJLk7=-vZ7_B%s4z)1}0C50=2?N99Nzal}85k86 z`P@|H^qws_qNFIGcu?#B1A{bB8#hE7*cc0t;-_H6f_!PJc6!sM91(I>6v)(36a*@_ z14;)yPfAJ<2zrt*Wzw^hgoFr#;%DyFjuOs7$2bxg7_@-~3PTL!;b0VE<6seBkzj$k z;xX9ZKt4C+I=!@H+hCw8Se2M0FkB%IG=vvo2pb235<>un3LB6^AfD0$dP-35+oMZM zp8Sz1R+MmV(r;j7z^Y)=5rhIx28K9>W+!$AhNQ$IAnC+#076PKFr*}w7y_{o5F4`s zgOmlR`0SY;d!Pb01_xjuyMdK907(Z11~A#c&%jVznv|TMV-8kT0O6MlwU$e}c9+PpjQ{_Yr{8UJ`Yym{-nlXRvT4GX`GVG>domAs8QX1nu!QTv ztLn4g#d5Y(%UQfwa`9cbRzPr`aPPPC-Ij|D#P5gC4^P=V*$H2hYm^{IB@j%!HlB^3s>}dGITW7pe6`5 zk>(X54->*3Jx=oU)G-ucns`h|$RW^`&rLJakBzO3)i}Lwi+GEnN+Qn!Avr!bMKeAz zMlnt?1^+3{7dz&6-;6jP!r|i4$gx8H%FIxp8RR~%*mjQMZPO0 zQYnP*2*XK6hA)f^q5Zag%!VQ@f4AR@e4n*B!qI3Mqqv5s*Cv)FxeeS3>ozcQsVFIJ z3MgR6x>d4eC$p&eBeomo`dtj?r`@wTdH%swuNyLZex;?=eX(2cPO6nH)n=mRl(V~z z-#ol1JH1%@-Msm36CaHBJ7`RIfYR?|QIbJNDzH;R@R` z9E$O$pT2*3@0v_bZ@i|%)|M45f_874`>MOHFBg_gU@pt|5O6O$`78RDR_q?X!|h+M t&G;1m{aW6x1gG+^m7m!;{xS+MN;D`aF){FH%wRNNZu-K^>>uPT0RT`YQ&<21 literal 0 HcmV?d00001 diff --git a/extensions-core/orc-extensions/example/test_2_hadoop_job.json b/extensions-core/orc-extensions/example/test_2_hadoop_job.json new file mode 100644 index 000000000000..30779f9040da --- /dev/null +++ b/extensions-core/orc-extensions/example/test_2_hadoop_job.json @@ -0,0 +1,70 @@ +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "example/test_2.orc" + }, + "metadataUpdateSpec": { + "type": "postgresql", + "connectURI": "jdbc:postgresql://localhost/druid", + "user" : "druid", + "password" : "asdf", + "segmentTable": "druid_segments" + }, + "segmentOutputPath": "/tmp/segments" + }, + "dataSchema": { + "dataSource": "test", + "parser": { + "type": "orc", + "parseSpec": { + "format": "orc", + "flattenSpec": { + "useFieldDiscovery": true, + "fields": [ + { + "type": "path", + "name": "col7-subcol7", + "expr": "$.col7.subcol7" + } + ] + }, + "timestampSpec": { + "column": "timestamp", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + ], + "dimensionExclusions": [], + "spatialDimensions": [] + } + } + }, + "metricsSpec": [], + "granularitySpec": { + "type": "uniform", + "segmentGranularity": "DAY", + "queryGranularity": "NONE", + "intervals": ["2015-01-01/2017-01-01"] + } + }, + "tuningConfig": { + "type": "hadoop", + "workingPath": "tmp/working_path", + "partitionsSpec": { + "targetPartitionSize": 5000000 + }, + "jobProperties" : { + "mapreduce.map.java.opts": "-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps", + "mapreduce.reduce.java.opts": "-server -Duser.timezone=UTC -Dfile.encoding=UTF-8 -XX:+PrintGCDetails -XX:+PrintGCTimeStamps", + "mapred.child.java.opts": "-server -XX:+PrintGCDetails -XX:+PrintGCTimeStamps" + }, + "leaveIntermediate": true + } + } +} diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml new file mode 100644 index 000000000000..9757182e9f4f --- /dev/null +++ b/extensions-core/orc-extensions/pom.xml @@ -0,0 +1,72 @@ + + + + org.apache.druid.extensions + druid-orc-extensions + druid-orc-extensions + druid-orc-extensions + + + druid + org.apache.druid + 0.15.0-incubating-SNAPSHOT + ../../pom.xml + + 4.0.0 + + 1.2.1 + + + + org.apache.druid + druid-indexing-hadoop + ${project.parent.version} + provided + + + org.apache.hadoop + hadoop-client + ${hadoop.compile.version} + provided + + + org.apache.orc + orc-mapreduce + ${orc.version} + + + com.esotericsoftware + kryo-shaded + + + commons-codec + commons-codec + + + + + junit + junit + test + + + diff --git a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java similarity index 96% rename from extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java rename to extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java index 595bb3b856e7..138f85e3aaac 100644 --- a/extensions-contrib/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcExtensionsModule.java @@ -36,7 +36,8 @@ public List getJacksonModules() return Collections.singletonList( new SimpleModule("OrcInputRowParserModule") .registerSubtypes( - new NamedType(OrcHadoopInputRowParser.class, "orc") + new NamedType(OrcHadoopInputRowParser.class, "orc"), + new NamedType(OrcParseSpec.class, "orc") ) ); } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java new file mode 100644 index 000000000000..ac1b1d4b0018 --- /dev/null +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.java.util.common.parsers.ObjectFlattener; +import org.apache.druid.java.util.common.parsers.ObjectFlatteners; +import org.apache.orc.mapred.OrcStruct; + +import javax.validation.constraints.NotNull; +import java.util.List; + +public class OrcHadoopInputRowParser implements InputRowParser +{ + private final ParseSpec parseSpec; + private final ObjectFlattener groupFlattener; + private final MapInputRowParser parser; + + @JsonCreator + public OrcHadoopInputRowParser( + @JsonProperty("parseSpec") ParseSpec parseSpec + ) + { + this.parseSpec = parseSpec; + final JSONPathSpec flattenSpec; + if ((parseSpec instanceof OrcParseSpec)) { + flattenSpec = ((OrcParseSpec) parseSpec).getFlattenSpec(); + } else { + flattenSpec = JSONPathSpec.DEFAULT; + } + this.groupFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker()); + this.parser = new MapInputRowParser(parseSpec); + } + + @NotNull + @Override + public List parseBatch(OrcStruct input) + { + return parser.parseBatch(groupFlattener.flatten(input)); + } + + @Override + public ParseSpec getParseSpec() + { + return parseSpec; + } + + @Override + public InputRowParser withParseSpec(ParseSpec parseSpec) + { + return new OrcHadoopInputRowParser(parseSpec); + } +} diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcParseSpec.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcParseSpec.java new file mode 100644 index 000000000000..6e46bfac318b --- /dev/null +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcParseSpec.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.NestedDataParseSpec; +import org.apache.druid.data.input.impl.ParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; + +public class OrcParseSpec extends NestedDataParseSpec +{ + @JsonCreator + public OrcParseSpec( + @JsonProperty("timestampSpec") TimestampSpec timestampSpec, + @JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec, + @JsonProperty("flattenSpec") JSONPathSpec flattenSpec + ) + { + super( + timestampSpec, + dimensionsSpec != null ? dimensionsSpec : DimensionsSpec.EMPTY, + flattenSpec != null ? flattenSpec : JSONPathSpec.DEFAULT + ); + } + + @Override + public ParseSpec withTimestampSpec(TimestampSpec spec) + { + return new OrcParseSpec(spec, getDimensionsSpec(), getFlattenSpec()); + } + + @Override + public ParseSpec withDimensionsSpec(DimensionsSpec spec) + { + return new OrcParseSpec(getTimestampSpec(), spec, getFlattenSpec()); + } + + @Override + public String toString() + { + return "OrcParseSpec{" + + "timestampSpec=" + getTimestampSpec() + + ", dimensionsSpec=" + getDimensionsSpec() + + ", flattenSpec=" + getFlattenSpec() + + "}"; + } +} diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java new file mode 100644 index 000000000000..abadb2ef5165 --- /dev/null +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import io.netty.util.SuppressForbidden; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.hadoop.hive.serde2.io.DateWritable; +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; +import org.apache.hadoop.io.BooleanWritable; +import org.apache.hadoop.io.ByteWritable; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.DoubleWritable; +import org.apache.hadoop.io.FloatWritable; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.ShortWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparable; +import org.apache.orc.TypeDescription; +import org.apache.orc.mapred.OrcList; +import org.apache.orc.mapred.OrcMap; +import org.apache.orc.mapred.OrcStruct; +import org.apache.orc.mapred.OrcTimestamp; +import org.joda.time.DateTime; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class OrcStructConverter +{ + /** + * Convert a orc struct field as though it were a map. Complex types will be transformed + * into java lists and maps when possible ({@link OrcStructConverter#convertList} and + * {@link OrcStructConverter#convertMap}), and + * primitive types will be extracted into an ingestion friendly state (e.g. 'int' and 'long'). Finally, + * if a field is not present, this method will return null. + * + * Note: "Union" types are not currently supported and will be returned as null + */ + @Nullable + Object convertField(OrcStruct struct, String fieldName) + { + TypeDescription schema = struct.getSchema(); + int fieldIndex = schema.getFieldNames().indexOf(fieldName); + + if (fieldIndex < 0) { + return null; + } + + TypeDescription fieldDescription = schema.getChildren().get(fieldIndex); + WritableComparable fieldValue = struct.getFieldValue(fieldIndex); + + if (fieldDescription.getCategory().isPrimitive()) { + return convertPrimitive(fieldDescription, fieldValue); + } else { + // handle complex column types + /* + ORC TYPE WRITABLE TYPE + array org.apache.orc.mapred.OrcList + map org.apache.orc.mapred.OrcMap + struct org.apache.orc.mapred.OrcStruct + uniontype org.apache.orc.mapred.OrcUnion + */ + switch (fieldDescription.getCategory()) { + case LIST: + OrcList orcList = (OrcList) fieldValue; + return convertList(fieldDescription, orcList); + case MAP: + OrcMap map = (OrcMap) fieldValue; + return convertMap(fieldDescription, map); + case STRUCT: + OrcStruct structMap = (OrcStruct) fieldValue; + return convertMap(structMap); + case UNION: + // sorry union types :( + default: + return null; + } + } + } + + @Nonnull + private List convertList(TypeDescription fieldDescription, OrcList orcList) + { + // if primitive list, convert primitives + TypeDescription listType = fieldDescription.getChildren().get(0); + if (listType.getCategory().isPrimitive()) { + return (List) orcList.stream() + .map(li -> convertPrimitive(listType, (WritableComparable) li)) + .collect(Collectors.toList()); + } + return new ArrayList(orcList); + } + + static Map convertMap( + TypeDescription fieldDescription, + OrcMap map + ) + { + Map converted = new HashMap<>(); + TypeDescription keyDescription = fieldDescription.getChildren().get(0); + TypeDescription valueDescription = fieldDescription.getChildren().get(1); + for (WritableComparable key : map.navigableKeySet()) { + Object newKey = convertPrimitive(keyDescription, key); + if (valueDescription.getCategory().isPrimitive()) { + converted.put(newKey, convertPrimitive(valueDescription, map.get(key))); + } else { + converted.put(newKey, map.get(key)); + } + } + return converted; + } + + private Map convertMap(OrcStruct map) + { + Map converted = new HashMap<>(); + for (String key : map.getSchema().getFieldNames()) { + converted.put(key, convertField(map, key)); + } + return converted; + } + + @SuppressForbidden(reason = "new DateTime(java.util.Date)") + private static Object convertPrimitive(TypeDescription fieldDescription, WritableComparable field) + { + /* + ORC TYPE WRITABLE TYPE + binary org.apache.hadoop.io.BytesWritable + bigint org.apache.hadoop.io.LongWritable + boolean org.apache.hadoop.io.BooleanWritable + char org.apache.hadoop.io.Text + date org.apache.hadoop.hive.serde2.io.DateWritable + decimal org.apache.hadoop.hive.serde2.io.HiveDecimalWritable + double org.apache.hadoop.io.DoubleWritable + float org.apache.hadoop.io.FloatWritable + int org.apache.hadoop.io.IntWritable + smallint org.apache.hadoop.io.ShortWritable + string org.apache.hadoop.io.Text + timestamp org.apache.orc.mapred.OrcTimestamp + tinyint org.apache.hadoop.io.ByteWritable + varchar org.apache.hadoop.io.Text + */ + switch (fieldDescription.getCategory()) { + case STRING: + case CHAR: + case VARCHAR: + return ((Text) field).toString(); + case BOOLEAN: + return ((BooleanWritable) field).get(); + case BYTE: + return ((ByteWritable) field).get(); + case SHORT: + return ((ShortWritable) field).get(); + case INT: + return ((IntWritable) field).get(); + case LONG: + return ((LongWritable) field).get(); + case FLOAT: + return ((FloatWritable) field).get(); + case DOUBLE: + return ((DoubleWritable) field).get(); + case DECIMAL: + return ((HiveDecimalWritable) field).getHiveDecimal().doubleValue(); + case TIMESTAMP: + return ((OrcTimestamp) field).getTime(); + case DATE: + // todo: is the the best way to go from java.util.Date to DateTime? + return new DateTime(((DateWritable) field).get()); + case BINARY: + // todo: hmm, i think we need a standard way of handling binary blobs this is a placeholder. + return StringUtils.encodeBase64String(((BytesWritable) field).getBytes()); + default: + return null; + } + } +} diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java new file mode 100644 index 000000000000..fb4f18070487 --- /dev/null +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import com.jayway.jsonpath.Configuration; +import com.jayway.jsonpath.JsonPath; +import com.jayway.jsonpath.Option; +import org.apache.druid.java.util.common.parsers.NotImplementedMappingProvider; +import org.apache.druid.java.util.common.parsers.ObjectFlatteners; +import org.apache.orc.TypeDescription; +import org.apache.orc.mapred.OrcList; +import org.apache.orc.mapred.OrcMap; +import org.apache.orc.mapred.OrcStruct; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; +import java.util.function.Function; + +public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker +{ + private final Configuration jsonPathConfiguration; + private final OrcStructConverter converter; + + OrcStructFlattenerMaker() + { + this.converter = new OrcStructConverter(); + this.jsonPathConfiguration = Configuration.builder() + .jsonProvider(new OrcStructJsonProvider(converter)) + .mappingProvider(new NotImplementedMappingProvider()) + .options(EnumSet.of(Option.SUPPRESS_EXCEPTIONS)) + .build(); + } + + @Override + public Iterable discoverRootFields(OrcStruct obj) + { + List fields = obj.getSchema().getFieldNames(); + List children = obj.getSchema().getChildren(); + List primitiveFields = new ArrayList<>(); + for (int i = 0; i < fields.size(); i++) { + if (children.get(i).getCategory().isPrimitive()) { + primitiveFields.add(fields.get(i)); + } + } + return primitiveFields; + } + + @Override + public Object getRootField(OrcStruct obj, String key) + { + return finalizeConversion(converter.convertField(obj, key)); + } + + @Override + public Function makeJsonPathExtractor(String expr) + { + final JsonPath jsonPath = JsonPath.compile(expr); + return record -> { + Object val = jsonPath.read(record, jsonPathConfiguration); + return finalizeConversion(val); + }; + } + + @Nullable + @Override + public Function makeJsonQueryExtractor(String expr) + { + throw new UnsupportedOperationException("ORC flattener does not support JQ"); + } + + private Object finalizeConversion(Object o) + { + // replace any remaining complex types with null + if (o instanceof OrcStruct || o instanceof OrcMap || o instanceof OrcList) { + return null; + } + return o; + } +} diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java new file mode 100644 index 000000000000..ed74bb5691bc --- /dev/null +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import com.jayway.jsonpath.InvalidJsonException; +import com.jayway.jsonpath.spi.json.JsonProvider; +import org.apache.orc.mapred.OrcStruct; + +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class OrcStructJsonProvider implements JsonProvider +{ + private final OrcStructConverter converter; + + OrcStructJsonProvider(OrcStructConverter converter) + { + this.converter = converter; + } + + @Override + public Object createArray() + { + return new ArrayList<>(); + } + + @Override + public Object createMap() + { + return new HashMap<>(); + } + + @Override + public boolean isArray(final Object o) + { + return o instanceof List; + } + + @Override + public boolean isMap(final Object o) + { + return o == null || o instanceof Map || o instanceof OrcStruct; + } + + @Override + public int length(final Object o) + { + if (o instanceof List) { + return ((List) o).size(); + } else { + return 0; + } + } + + @Override + public Iterable toIterable(final Object o) + { + if (o instanceof List) { + return (List) o; + } + throw new UnsupportedOperationException(o.getClass().getName()); + } + + @Override + public Collection getPropertyKeys(final Object o) + { + if (o == null) { + return Collections.emptySet(); + } else if (o instanceof Map) { + return ((Map) o).keySet().stream().map(String::valueOf).collect(Collectors.toSet()); + } else if (o instanceof OrcStruct) { + return ((OrcStruct) o).getSchema().getFieldNames(); + } else { + throw new UnsupportedOperationException(o.getClass().getName()); + } + } + + @Override + public Object getMapValue(final Object o, final String s) + { + if (o == null) { + return null; + } else if (o instanceof Map) { + return ((Map) o).get(s); + } else if (o instanceof OrcStruct) { + OrcStruct g = (OrcStruct) o; + return converter.convertField(g, s); + } + throw new UnsupportedOperationException(o.getClass().getName()); + } + + @Override + public Object getArrayIndex(final Object o, final int i) + { + if (o instanceof List) { + return ((List) o).get(i); + } + throw new UnsupportedOperationException(o.getClass().getName()); + } + + @Override + public void setArrayIndex(final Object o, final int i, final Object o1) + { + if (o instanceof List) { + final List list = (List) o; + if (list.size() == i) { + list.add(o1); + } else { + list.set(i, o1); + } + } else { + throw new UnsupportedOperationException(o.getClass().getName()); + } + } + + @Override + public void setProperty(final Object o, final Object o1, final Object o2) + { + if (o instanceof Map) { + ((Map) o).put(o1, o2); + } else { + throw new UnsupportedOperationException(o.getClass().getName()); + } + } + + @Override + public void removeProperty(final Object o, final Object o1) + { + if (o instanceof Map) { + ((Map) o).remove(o1); + } else { + throw new UnsupportedOperationException(o.getClass().getName()); + } + } + + @Override + @Deprecated + public Object getArrayIndex(final Object o, final int i, final boolean b) + { + throw new UnsupportedOperationException("Deprecated"); + } + + @Override + public Object parse(final String s) throws InvalidJsonException + { + throw new UnsupportedOperationException("Unused"); + } + + @Override + public Object parse(final InputStream inputStream, final String s) throws InvalidJsonException + { + throw new UnsupportedOperationException("Unused"); + } + + @Override + public String toJson(final Object o) + { + throw new UnsupportedOperationException("Unused"); + } + + @Override + public Object unwrap(final Object o) + { + throw new UnsupportedOperationException("Unused"); + } +} diff --git a/extensions-contrib/orc-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-core/orc-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule similarity index 100% rename from extensions-contrib/orc-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule rename to extensions-core/orc-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java new file mode 100644 index 000000000000..c9103e441e1a --- /dev/null +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java @@ -0,0 +1,263 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.data.input.orc; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.impl.InputRowParser; +import org.apache.druid.indexer.HadoopDruidIndexerConfig; +import org.apache.druid.indexer.path.StaticPathSpec; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.orc.mapred.OrcStruct; +import org.apache.orc.mapreduce.OrcInputFormat; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class OrcHadoopInputRowParserTest +{ + @Test + public void testTest1() throws IOException, InterruptedException + { + HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/test_1_hadoop_job.json"); + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + /* + test_1.orc + struct,val1:float> + {2016-01-01T00:00:00.000Z, bar, [dat1, dat2, dat3], 1.1} + */ + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("bar", rows.get(0).getDimension("col1").get(0)); + String s1 = rows.get(0).getDimension("col2").get(0); + String s2 = rows.get(0).getDimension("col2").get(1); + String s3 = rows.get(0).getDimension("col2").get(2); + assertEquals("dat1", s1); + assertEquals("dat2", s2); + assertEquals("dat3", s3); + } + + @Test + public void testTest2() throws IOException, InterruptedException + { + HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/test_2_hadoop_job.json"); + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + /* + test_2.orc + struct,col3:float,col4:bigint,col5:decimal,col6:array,col7:map> + {2016-01-01, bar, [dat1, dat2, dat3], 1.1, 2, 3.5, [], {subcol7=subval7}} + */ + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("bar", rows.get(0).getDimension("col1").get(0)); + assertEquals("dat1", rows.get(0).getDimension("col2").get(0)); + assertEquals("dat2", rows.get(0).getDimension("col2").get(1)); + assertEquals("dat3", rows.get(0).getDimension("col2").get(2)); + assertEquals(1.1f, rows.get(0).getRaw("col3")); + assertEquals(2L, rows.get(0).getRaw("col4")); + assertEquals(3.5d, rows.get(0).getRaw("col5")); + assertEquals(ImmutableList.of(), rows.get(0).getRaw("col6")); + assertEquals("subval7", rows.get(0).getRaw("col7-subcol7")); + } + + @Test + public void testOrcFile11Format() throws IOException, InterruptedException + { + // not sure what file 11 format means, but we'll test it! + + /* + orc-file-11-format.orc + struct>>,list:array>,map:map>,ts:timestamp,decimal1:decimal(38,10)> + {false, 1, 1024, 65536, 9223372036854775807, 1.0, -15.0, 00 01 02 03 04, hi, {[{1, bye}, {2, sigh}]}, [{3, good}, {4, bad}], {}, 2000-03-12 15:00:00.0, 12345678.6547456} + */ + HadoopDruidIndexerConfig config = + loadHadoopDruidIndexerConfig("example/orc-file-11-format-hadoop-job.json"); + + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("false", rows.get(0).getDimension("boolean1").get(0)); + assertEquals("1", rows.get(0).getDimension("byte1").get(0)); + assertEquals("1024", rows.get(0).getDimension("short1").get(0)); + assertEquals("65536", rows.get(0).getDimension("int1").get(0)); + assertEquals("9223372036854775807", rows.get(0).getDimension("long1").get(0)); + assertEquals("1.0", rows.get(0).getDimension("float1").get(0)); + assertEquals("-15.0", rows.get(0).getDimension("double1").get(0)); + assertEquals("AAECAwQAAA==", rows.get(0).getDimension("bytes1").get(0)); + assertEquals("hi", rows.get(0).getDimension("string1").get(0)); + assertEquals("1.23456786547456E7", rows.get(0).getDimension("decimal1").get(0)); + assertEquals("2", rows.get(0).getDimension("struct_list_struct_int").get(0)); + assertEquals("1", rows.get(0).getDimension("struct_list_struct_intlist").get(0)); + assertEquals("2", rows.get(0).getDimension("struct_list_struct_intlist").get(1)); + assertEquals("good", rows.get(0).getDimension("list_struct_string").get(0)); + assertEquals(DateTimes.of("2000-03-12T15:00:00.0Z"), rows.get(0).getTimestamp()); + + // first row has empty 'map' column, so lets read another! + List allRows = getAllRows(config); + InputRow anotherRow = allRows.get(0); + assertEquals("true", anotherRow.getDimension("boolean1").get(0)); + assertEquals("100", anotherRow.getDimension("byte1").get(0)); + assertEquals("2048", anotherRow.getDimension("short1").get(0)); + assertEquals("65536", anotherRow.getDimension("int1").get(0)); + assertEquals("9223372036854775807", anotherRow.getDimension("long1").get(0)); + assertEquals("2.0", anotherRow.getDimension("float1").get(0)); + assertEquals("-5.0", anotherRow.getDimension("double1").get(0)); + assertEquals("AAECAwQAAA==", rows.get(0).getDimension("bytes1").get(0)); + assertEquals("bye", anotherRow.getDimension("string1").get(0)); + assertEquals("1.23456786547457E7", anotherRow.getDimension("decimal1").get(0)); + assertEquals("2", anotherRow.getDimension("struct_list_struct_int").get(0)); + assertEquals("cat", anotherRow.getDimension("list_struct_string").get(0)); + assertEquals("5", anotherRow.getDimension("map_struct_int").get(0)); + } + + @Test + public void testOrcSplitElim() throws IOException, InterruptedException + { + // not sure what SplitElim means, but we'll test it! + + /* + orc_split_elim.orc + struct + {2, foo, 0.8, 1.2, 1969-12-31 16:00:00.0} + */ + HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/orc_split_elim_hadoop_job.json"); + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("2", rows.get(0).getDimension("userid").get(0)); + assertEquals("foo", rows.get(0).getDimension("string1").get(0)); + assertEquals("0.8", rows.get(0).getDimension("subtype").get(0)); + assertEquals("1.2", rows.get(0).getDimension("decimal1").get(0)); + assertEquals(DateTimes.of("1969-12-31T16:00:00.0Z"), rows.get(0).getTimestamp()); + } + + @Test + public void testDate1900() throws IOException, InterruptedException + { + /* + TestOrcFile.testDate1900.orc + struct + {1900-05-05 12:34:56.1, 1900-12-25} + */ + HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/testDate1900_hadoop_job.json"); + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("1900-12-25T00:00:00.000Z", rows.get(0).getDimension("date").get(0)); + assertEquals(DateTimes.of("1900-05-05T12:34:56.1Z"), rows.get(0).getTimestamp()); + } + + @Test + public void testDate2038() throws IOException, InterruptedException + { + /* + TestOrcFile.testDate2038.orc + struct + {2038-05-05 12:34:56.1, 2038-12-25} + */ + HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/testDate2038_hadoop_job.json"); + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + + OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); + List rows = (List) config.getParser().parseBatch(data); + assertEquals("2038-12-25T00:00:00.000Z", rows.get(0).getDimension("date").get(0)); + assertEquals(DateTimes.of("2038-05-05T12:34:56.1Z"), rows.get(0).getTimestamp()); + } + + private static HadoopDruidIndexerConfig loadHadoopDruidIndexerConfig(String configPath) + { + return HadoopDruidIndexerConfig.fromFile(new File(configPath)); + } + + private static OrcStruct getFirstRow(Job job, String orcPath) throws IOException, InterruptedException + { + File testFile = new File(orcPath); + Path path = new Path(testFile.getAbsoluteFile().toURI()); + FileSplit split = new FileSplit(path, 0, testFile.length(), null); + + InputFormat inputFormat = ReflectionUtils.newInstance( + OrcInputFormat.class, + job.getConfiguration() + ); + TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); + + try (RecordReader reader = inputFormat.createRecordReader(split, context)) { + + reader.initialize(split, context); + reader.nextKeyValue(); + return (OrcStruct) reader.getCurrentValue(); + } + } + + private static List getAllRows(HadoopDruidIndexerConfig config) + throws IOException, InterruptedException + { + Job job = Job.getInstance(new Configuration()); + config.intoConfiguration(job); + + File testFile = new File(((StaticPathSpec) config.getPathSpec()).getPaths()); + Path path = new Path(testFile.getAbsoluteFile().toURI()); + FileSplit split = new FileSplit(path, 0, testFile.length(), null); + + InputFormat inputFormat = ReflectionUtils.newInstance( + OrcInputFormat.class, + job.getConfiguration() + ); + TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); + + try (RecordReader reader = inputFormat.createRecordReader(split, context)) { + List records = new ArrayList<>(); + InputRowParser parser = config.getParser(); + + reader.initialize(split, context); + while (reader.nextKeyValue()) { + reader.nextKeyValue(); + Object data = reader.getCurrentValue(); + records.add(((List) parser.parseBatch(data)).get(0)); + } + + return records; + } + } +} diff --git a/extensions-contrib/orc-extensions/src/test/resources/log4j2.xml b/extensions-core/orc-extensions/src/test/resources/log4j2.xml similarity index 100% rename from extensions-contrib/orc-extensions/src/test/resources/log4j2.xml rename to extensions-core/orc-extensions/src/test/resources/log4j2.xml diff --git a/pom.xml b/pom.xml index 1ab8ad1b6c42..60cbc78bb043 100644 --- a/pom.xml +++ b/pom.xml @@ -137,6 +137,7 @@ extensions-core/kafka-indexing-service extensions-core/kinesis-indexing-service extensions-core/mysql-metadata-storage + extensions-core/orc-extensions extensions-core/parquet-extensions extensions-core/postgresql-metadata-storage extensions-core/protobuf-extensions @@ -156,7 +157,6 @@ extensions-contrib/rabbitmq extensions-contrib/distinctcount extensions-contrib/statsd-emitter - extensions-contrib/orc-extensions extensions-contrib/time-min-max extensions-contrib/google-extensions extensions-contrib/virtual-columns From 04dbefeeaf28c30f89fd474bfe852cade9ac1288 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 12 Mar 2019 01:29:23 -0700 Subject: [PATCH 02/13] change binary handling to be compatible with avro and parquet, Rows.objectToStrings now converts byte[] to base64, change date handling --- .../org/apache/druid/data/input/Rows.java | 3 + extensions-core/orc-extensions/pom.xml | 6 + .../input/orc/OrcHadoopInputRowParser.java | 10 +- .../data/input/orc/OrcStructConverter.java | 164 ++++++++++-------- .../input/orc/OrcStructFlattenerMaker.java | 4 +- 5 files changed, 105 insertions(+), 82 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/Rows.java b/core/src/main/java/org/apache/druid/data/input/Rows.java index 8514805ca93d..7435272a1886 100644 --- a/core/src/main/java/org/apache/druid/data/input/Rows.java +++ b/core/src/main/java/org/apache/druid/data/input/Rows.java @@ -66,6 +66,9 @@ public static List objectToStrings(final Object inputValue) } else if (inputValue instanceof List) { // guava's toString function fails on null objects, so please do not use it return ((List) inputValue).stream().map(String::valueOf).collect(Collectors.toList()); + } else if (inputValue instanceof byte[]) { + // convert byte[] to base64 encoded string + return Collections.singletonList(StringUtils.encodeBase64String((byte[]) inputValue)); } else { return Collections.singletonList(String.valueOf(inputValue)); } diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index 9757182e9f4f..42b0bcc7aa0b 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -36,6 +36,12 @@ 1.2.1 + + org.apache.druid + druid-core + ${project.parent.version} + provided + org.apache.druid druid-indexing-hadoop diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java index ac1b1d4b0018..be179fb49834 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.parsers.ObjectFlatteners; import org.apache.orc.mapred.OrcStruct; +import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.util.List; @@ -38,20 +39,23 @@ public class OrcHadoopInputRowParser implements InputRowParser private final ParseSpec parseSpec; private final ObjectFlattener groupFlattener; private final MapInputRowParser parser; + private final boolean binaryAsString; @JsonCreator public OrcHadoopInputRowParser( - @JsonProperty("parseSpec") ParseSpec parseSpec + @JsonProperty("parseSpec") ParseSpec parseSpec, + @Nullable @JsonProperty("binaryAsString") Boolean binaryAsString ) { this.parseSpec = parseSpec; + this.binaryAsString = binaryAsString == null ? false : binaryAsString; final JSONPathSpec flattenSpec; if ((parseSpec instanceof OrcParseSpec)) { flattenSpec = ((OrcParseSpec) parseSpec).getFlattenSpec(); } else { flattenSpec = JSONPathSpec.DEFAULT; } - this.groupFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker()); + this.groupFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(false)); this.parser = new MapInputRowParser(parseSpec); } @@ -71,6 +75,6 @@ public ParseSpec getParseSpec() @Override public InputRowParser withParseSpec(ParseSpec parseSpec) { - return new OrcHadoopInputRowParser(parseSpec); + return new OrcHadoopInputRowParser(parseSpec, binaryAsString); } } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java index abadb2ef5165..e458af640561 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -19,7 +19,7 @@ package org.apache.druid.data.input.orc; -import io.netty.util.SuppressForbidden; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable; @@ -38,7 +38,6 @@ import org.apache.orc.mapred.OrcMap; import org.apache.orc.mapred.OrcStruct; import org.apache.orc.mapred.OrcTimestamp; -import org.joda.time.DateTime; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -50,6 +49,89 @@ public class OrcStructConverter { + static Map convertMap( + TypeDescription fieldDescription, + OrcMap map, + boolean binaryAsString + ) + { + Map converted = new HashMap<>(); + TypeDescription keyDescription = fieldDescription.getChildren().get(0); + TypeDescription valueDescription = fieldDescription.getChildren().get(1); + for (WritableComparable key : map.navigableKeySet()) { + Object newKey = convertPrimitive(keyDescription, key, binaryAsString); + if (valueDescription.getCategory().isPrimitive()) { + converted.put(newKey, convertPrimitive(valueDescription, map.get(key), binaryAsString)); + } else { + converted.put(newKey, map.get(key)); + } + } + return converted; + } + + private static Object convertPrimitive(TypeDescription fieldDescription, WritableComparable field, boolean binaryAsString) + { + /* + ORC TYPE WRITABLE TYPE + binary org.apache.hadoop.io.BytesWritable + bigint org.apache.hadoop.io.LongWritable + boolean org.apache.hadoop.io.BooleanWritable + char org.apache.hadoop.io.Text + date org.apache.hadoop.hive.serde2.io.DateWritable + decimal org.apache.hadoop.hive.serde2.io.HiveDecimalWritable + double org.apache.hadoop.io.DoubleWritable + float org.apache.hadoop.io.FloatWritable + int org.apache.hadoop.io.IntWritable + smallint org.apache.hadoop.io.ShortWritable + string org.apache.hadoop.io.Text + timestamp org.apache.orc.mapred.OrcTimestamp + tinyint org.apache.hadoop.io.ByteWritable + varchar org.apache.hadoop.io.Text + */ + switch (fieldDescription.getCategory()) { + case STRING: + case CHAR: + case VARCHAR: + return ((Text) field).toString(); + case BOOLEAN: + return ((BooleanWritable) field).get(); + case BYTE: + return ((ByteWritable) field).get(); + case SHORT: + return ((ShortWritable) field).get(); + case INT: + return ((IntWritable) field).get(); + case LONG: + return ((LongWritable) field).get(); + case FLOAT: + return ((FloatWritable) field).get(); + case DOUBLE: + return ((DoubleWritable) field).get(); + case DECIMAL: + return ((HiveDecimalWritable) field).getHiveDecimal().doubleValue(); + case TIMESTAMP: + return ((OrcTimestamp) field).getTime(); + case DATE: + return DateTimes.utc(((DateWritable) field).get().getTime()); + case BINARY: + byte[] bytes = ((BytesWritable) field).getBytes(); + if (binaryAsString) { + return StringUtils.fromUtf8(bytes); + } else { + return bytes; + } + default: + return null; + } + } + + private boolean binaryAsString; + + OrcStructConverter(boolean binaryAsString) + { + this.binaryAsString = binaryAsString; + } + /** * Convert a orc struct field as though it were a map. Complex types will be transformed * into java lists and maps when possible ({@link OrcStructConverter#convertList} and @@ -73,7 +155,7 @@ Object convertField(OrcStruct struct, String fieldName) WritableComparable fieldValue = struct.getFieldValue(fieldIndex); if (fieldDescription.getCategory().isPrimitive()) { - return convertPrimitive(fieldDescription, fieldValue); + return convertPrimitive(fieldDescription, fieldValue, binaryAsString); } else { // handle complex column types /* @@ -89,7 +171,7 @@ Object convertField(OrcStruct struct, String fieldName) return convertList(fieldDescription, orcList); case MAP: OrcMap map = (OrcMap) fieldValue; - return convertMap(fieldDescription, map); + return convertMap(fieldDescription, map, binaryAsString); case STRUCT: OrcStruct structMap = (OrcStruct) fieldValue; return convertMap(structMap); @@ -108,30 +190,12 @@ private List convertList(TypeDescription fieldDescription, OrcList orcLi TypeDescription listType = fieldDescription.getChildren().get(0); if (listType.getCategory().isPrimitive()) { return (List) orcList.stream() - .map(li -> convertPrimitive(listType, (WritableComparable) li)) + .map(li -> convertPrimitive(listType, (WritableComparable) li, binaryAsString)) .collect(Collectors.toList()); } return new ArrayList(orcList); } - static Map convertMap( - TypeDescription fieldDescription, - OrcMap map - ) - { - Map converted = new HashMap<>(); - TypeDescription keyDescription = fieldDescription.getChildren().get(0); - TypeDescription valueDescription = fieldDescription.getChildren().get(1); - for (WritableComparable key : map.navigableKeySet()) { - Object newKey = convertPrimitive(keyDescription, key); - if (valueDescription.getCategory().isPrimitive()) { - converted.put(newKey, convertPrimitive(valueDescription, map.get(key))); - } else { - converted.put(newKey, map.get(key)); - } - } - return converted; - } private Map convertMap(OrcStruct map) { @@ -141,58 +205,4 @@ private Map convertMap(OrcStruct map) } return converted; } - - @SuppressForbidden(reason = "new DateTime(java.util.Date)") - private static Object convertPrimitive(TypeDescription fieldDescription, WritableComparable field) - { - /* - ORC TYPE WRITABLE TYPE - binary org.apache.hadoop.io.BytesWritable - bigint org.apache.hadoop.io.LongWritable - boolean org.apache.hadoop.io.BooleanWritable - char org.apache.hadoop.io.Text - date org.apache.hadoop.hive.serde2.io.DateWritable - decimal org.apache.hadoop.hive.serde2.io.HiveDecimalWritable - double org.apache.hadoop.io.DoubleWritable - float org.apache.hadoop.io.FloatWritable - int org.apache.hadoop.io.IntWritable - smallint org.apache.hadoop.io.ShortWritable - string org.apache.hadoop.io.Text - timestamp org.apache.orc.mapred.OrcTimestamp - tinyint org.apache.hadoop.io.ByteWritable - varchar org.apache.hadoop.io.Text - */ - switch (fieldDescription.getCategory()) { - case STRING: - case CHAR: - case VARCHAR: - return ((Text) field).toString(); - case BOOLEAN: - return ((BooleanWritable) field).get(); - case BYTE: - return ((ByteWritable) field).get(); - case SHORT: - return ((ShortWritable) field).get(); - case INT: - return ((IntWritable) field).get(); - case LONG: - return ((LongWritable) field).get(); - case FLOAT: - return ((FloatWritable) field).get(); - case DOUBLE: - return ((DoubleWritable) field).get(); - case DECIMAL: - return ((HiveDecimalWritable) field).getHiveDecimal().doubleValue(); - case TIMESTAMP: - return ((OrcTimestamp) field).getTime(); - case DATE: - // todo: is the the best way to go from java.util.Date to DateTime? - return new DateTime(((DateWritable) field).get()); - case BINARY: - // todo: hmm, i think we need a standard way of handling binary blobs this is a placeholder. - return StringUtils.encodeBase64String(((BytesWritable) field).getBytes()); - default: - return null; - } - } } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index fb4f18070487..71c9f997202b 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -40,9 +40,9 @@ public class OrcStructFlattenerMaker implements ObjectFlatteners.FlattenerMaker< private final Configuration jsonPathConfiguration; private final OrcStructConverter converter; - OrcStructFlattenerMaker() + OrcStructFlattenerMaker(boolean binaryAsString) { - this.converter = new OrcStructConverter(); + this.converter = new OrcStructConverter(binaryAsString); this.jsonPathConfiguration = Configuration.builder() .jsonProvider(new OrcStructJsonProvider(converter)) .mappingProvider(new NotImplementedMappingProvider()) From 58a63cd72d4dda0f274a6b447bdd18501b9308e3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 22 Mar 2019 00:02:46 -0700 Subject: [PATCH 03/13] better docs and tests --- .../development/extensions-core/orc.md | 155 +++++++++++++++++- .../example/test_1_hadoop_job.json | 10 -- .../input/orc/OrcStructFlattenerMaker.java | 3 +- .../orc/OrcHadoopInputRowParserTest.java | 8 + 4 files changed, 158 insertions(+), 18 deletions(-) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index 9e75c0ac9bc9..cc33cdfd8f26 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -37,17 +37,26 @@ The `inputFormat` of `inputSpec` in `ioConfig` must be set to `"org.apache.orc.m |Field | Type | Description | Required| |----------|-------------|----------------------------------------------------------------------------------------|---------| |type | String | This should say `orc` | yes| -|parseSpec | JSON Object | Specifies the timestamp and dimensions of the data. Any parse spec that extends ParseSpec is possible but only their TimestampSpec and DimensionsSpec are used. | yes| +|parseSpec | JSON Object | Specifies the timestamp and dimensions of the data (`timeAndDims` and `orc` format) and a `flattenSpec` (`orc` format) | yes| -The parser support auto field discovery and flattening if provided with a -[flattenSpec](../../ingestion/flatten-json.html) with `orc` as the `format`. -[All column types](https://orc.apache.org/docs/types.html) with the exception of `union` types are supported. Columns of - `list` type if filled with primitives may be used as a multi-value dimension, or specific elements can be extracted with +The parser supports two `parseSpec` formats: `orc` and `timeAndDims`. + +`orc` supports auto field discovery and flattening, if specified with a [flattenSpec](../../ingestion/flatten-json.html). +If no `flattenSpec` is specified, `useFieldDiscovery` will be enabled by default. Specifying a `dimensionSpec` is +optional if `useFieldDiscovery` is enabled: if a `dimensionSpec` is supplied, the list of `dimensions` it defines will be +the set of ingested dimensions, if missing the discovered fields will make up the list. + +`timeAndDims` parse spec must specify which fields will be extracted as dimensions through the `dimensionSpec`. + +[All column types](https://orc.apache.org/docs/types.html) are supported, with the exception of `union` types. Columns of + `list` type, if filled with primitives, may be used as a multi-value dimension, or specific elements can be extracted with `flattenSpec` expressions. Likewise, primitive fields may be extracted from `map` and `struct` types in the same manner. +Auto field discovery will automatically create a string dimension for every (non-timestamp) primitive or `list` of +primitives, as well as any flatten expressions defined in the `flattenSpec`. ### Examples -#### `orc` parser, `orc` parseSpec +#### `orc` parser, `orc` parseSpec, auto field discovery, flatten expressions ```json { @@ -83,12 +92,102 @@ The parser support auto field discovery and flattening if provided with a } ] }, + "timestampSpec": { + "column": "timestamp", + "format": "millis" + } + } + }, + ... + }, + "tuningConfig": + } + } +} +``` + +#### `orc` parser, `orc` parseSpec, field discovery with no flattenSpec or dimensionSpec + +```json +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "path/to/file.orc" + }, + ... + }, + "dataSchema": { + "dataSource": "example", + "parser": { + "type": "orc", + "parseSpec": { + "format": "orc", + "timestampSpec": { + "column": "timestamp", + "format": "millis" + } + } + }, + ... + }, + "tuningConfig": + } + } +} +``` + +#### `orc` parser, `orc` parseSpec, no autodiscovery + +```json +{ + "type": "index_hadoop", + "spec": { + "ioConfig": { + "type": "hadoop", + "inputSpec": { + "type": "static", + "inputFormat": "org.apache.orc.mapreduce.OrcInputFormat", + "paths": "path/to/file.orc" + }, + ... + }, + "dataSchema": { + "dataSource": "example", + "parser": { + "type": "orc", + "parseSpec": { + "format": "orc", + "flattenSpec": { + "useFieldDiscovery": false, + "fields": [ + { + "type": "path", + "name": "nestedDim", + "expr": "$.nestedData.dim1" + }, + { + "type": "path", + "name": "listDimFirstItem", + "expr": "$.listDim[1]" + } + ] + }, "timestampSpec": { "column": "timestamp", "format": "millis" }, "dimensionsSpec": { - "dimensions": [], + "dimensions": [ + "dim1", + "dim3", + "nestedDim", + "listDimFirstItem" + ], "dimensionExclusions": [], "spatialDimensions": [] } @@ -145,3 +244,45 @@ The parser support auto field discovery and flattening if provided with a } ``` + +### Migration from 'contrib' extension +This extension, first available in version 0.15.0, replaces the previous 'contrib' extension which was available until +0.14.0-incubating. While this extension can index any data the 'contrib' extension could, the json spec for the +ingestion task is *incompatible*, and will need modified to work with the newer 'core' extension. + +To migrate to 0.15.0+: +* In `inputSpec` of `ioConfig`, `inputFormat` must be changed from `"org.apache.hadoop.hive.ql.io.orc.OrcNewInputFormat"` to +`"org.apache.orc.mapreduce.OrcInputFormat"` +* The 'contrib' extension supported a `typeString` property, which provided the schema of the +ORC file, of which was essentially required to have the types correct, but notably _not_ the column names, which +facilitated column renaming. In the 'core' extension, column renaming can be achieved with +[`flattenSpec` expressions](../../ingestion/flatten-json.html). For example, `"typeString":"struct"` +with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid schema would need replaced with: +```json +"flattenSpec": { + "fields": [ + { + "type": "path", + "name": "name", + "expr": "$._col1" + } + ] + ... +} +``` +* The 'contrib' extension supported a `mapFieldNameFormat` property, which provided a way to specify a dimension to + flatten `OrcMap` columns with primitive type keys. This functionality has also been replaced with + [`flattenSpec` expressions](../../ingestion/flatten-json.html). For example: `"mapFieldNameFormat": "_"` + for a dimension `nestedData_dim1`, to preserve Druid schema could be replaced with + ```json +"flattenSpec": { + "fields": [ + { + "type": "path", + "name": "nestedData_dim1", + "expr": "$.nestedData.dim1" + } + ] + ... +} +``` \ No newline at end of file diff --git a/extensions-core/orc-extensions/example/test_1_hadoop_job.json b/extensions-core/orc-extensions/example/test_1_hadoop_job.json index df65ffc73bed..1ef0c4ed0dce 100755 --- a/extensions-core/orc-extensions/example/test_1_hadoop_job.json +++ b/extensions-core/orc-extensions/example/test_1_hadoop_job.json @@ -23,19 +23,9 @@ "type": "orc", "parseSpec": { "format": "orc", - "flattenSpec": { - "useFieldDiscovery": true, - "fields": [] - }, "timestampSpec": { "column": "timestamp", "format": "auto" - }, - "dimensionsSpec": { - "dimensions": [ - ], - "dimensionExclusions": [], - "spatialDimensions": [] } } }, diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index 71c9f997202b..2171b6c3726a 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -57,7 +57,8 @@ public Iterable discoverRootFields(OrcStruct obj) List children = obj.getSchema().getChildren(); List primitiveFields = new ArrayList<>(); for (int i = 0; i < fields.size(); i++) { - if (children.get(i).getCategory().isPrimitive()) { + if (children.get(i).getCategory().isPrimitive() || (children.get(i).getCategory().equals(TypeDescription.Category.LIST) && + children.get(i).getChildren().get(0).getCategory().isPrimitive())) { primitiveFields.add(fields.get(i)); } } diff --git a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java index c9103e441e1a..8dd399fa0dd1 100644 --- a/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java +++ b/extensions-core/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java @@ -51,6 +51,7 @@ public class OrcHadoopInputRowParserTest @Test public void testTest1() throws IOException, InterruptedException { + // total auto-discover fields (no flattenSpec, no dimensionSpec) HadoopDruidIndexerConfig config = loadHadoopDruidIndexerConfig("example/test_1_hadoop_job.json"); Job job = Job.getInstance(new Configuration()); config.intoConfiguration(job); @@ -61,6 +62,7 @@ public void testTest1() throws IOException, InterruptedException */ OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(3, rows.get(0).getDimensions().size()); assertEquals("bar", rows.get(0).getDimension("col1").get(0)); String s1 = rows.get(0).getDimension("col2").get(0); String s2 = rows.get(0).getDimension("col2").get(1); @@ -83,6 +85,7 @@ public void testTest2() throws IOException, InterruptedException */ OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(7, rows.get(0).getDimensions().size()); assertEquals("bar", rows.get(0).getDimension("col1").get(0)); assertEquals("dat1", rows.get(0).getDimension("col2").get(0)); assertEquals("dat2", rows.get(0).getDimension("col2").get(1)); @@ -112,6 +115,7 @@ public void testOrcFile11Format() throws IOException, InterruptedException OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(14, rows.get(0).getDimensions().size()); assertEquals("false", rows.get(0).getDimension("boolean1").get(0)); assertEquals("1", rows.get(0).getDimension("byte1").get(0)); assertEquals("1024", rows.get(0).getDimension("short1").get(0)); @@ -131,6 +135,7 @@ public void testOrcFile11Format() throws IOException, InterruptedException // first row has empty 'map' column, so lets read another! List allRows = getAllRows(config); InputRow anotherRow = allRows.get(0); + assertEquals(14, rows.get(0).getDimensions().size()); assertEquals("true", anotherRow.getDimension("boolean1").get(0)); assertEquals("100", anotherRow.getDimension("byte1").get(0)); assertEquals("2048", anotherRow.getDimension("short1").get(0)); @@ -162,6 +167,7 @@ public void testOrcSplitElim() throws IOException, InterruptedException OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(4, rows.get(0).getDimensions().size()); assertEquals("2", rows.get(0).getDimension("userid").get(0)); assertEquals("foo", rows.get(0).getDimension("string1").get(0)); assertEquals("0.8", rows.get(0).getDimension("subtype").get(0)); @@ -183,6 +189,7 @@ public void testDate1900() throws IOException, InterruptedException OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(1, rows.get(0).getDimensions().size()); assertEquals("1900-12-25T00:00:00.000Z", rows.get(0).getDimension("date").get(0)); assertEquals(DateTimes.of("1900-05-05T12:34:56.1Z"), rows.get(0).getTimestamp()); } @@ -201,6 +208,7 @@ public void testDate2038() throws IOException, InterruptedException OrcStruct data = getFirstRow(job, ((StaticPathSpec) config.getPathSpec()).getPaths()); List rows = (List) config.getParser().parseBatch(data); + assertEquals(1, rows.get(0).getDimensions().size()); assertEquals("2038-12-25T00:00:00.000Z", rows.get(0).getDimension("date").get(0)); assertEquals(DateTimes.of("2038-05-05T12:34:56.1Z"), rows.get(0).getTimestamp()); } From c674c4da4dda70cea9e0018992052c48ace4b307 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 22 Mar 2019 00:18:53 -0700 Subject: [PATCH 04/13] fix it --- docs/content/development/extensions-core/orc.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index cc33cdfd8f26..1c7ca4a609b4 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -261,6 +261,11 @@ with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid sc ```json "flattenSpec": { "fields": [ + { + "type": "path", + "name": "time", + "expr": "$._col0" + }, { "type": "path", "name": "name", From 90ff75d22db6bf3c9e03d2abbe5a8289281c1c69 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 22 Mar 2019 00:19:25 -0700 Subject: [PATCH 05/13] formatting --- docs/content/development/extensions-core/orc.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index 1c7ca4a609b4..1619c4564741 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -262,9 +262,9 @@ with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid sc "flattenSpec": { "fields": [ { - "type": "path", - "name": "time", - "expr": "$._col0" + "type": "path", + "name": "time", + "expr": "$._col0" }, { "type": "path", From e032ed3cfac2dd049fdaa2a0e6057359658004d8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 22 Mar 2019 00:20:56 -0700 Subject: [PATCH 06/13] doc fix --- docs/content/development/extensions-core/orc.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index 1619c4564741..3d7559ca66a8 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -276,7 +276,7 @@ with the actual schema `struct<_col0:string,_col1:string>`, to preserve Druid sc } ``` * The 'contrib' extension supported a `mapFieldNameFormat` property, which provided a way to specify a dimension to - flatten `OrcMap` columns with primitive type keys. This functionality has also been replaced with + flatten `OrcMap` columns with primitive types. This functionality has also been replaced with [`flattenSpec` expressions](../../ingestion/flatten-json.html). For example: `"mapFieldNameFormat": "_"` for a dimension `nestedData_dim1`, to preserve Druid schema could be replaced with ```json From c552eb54fa0af35abbdc1782eca1dc6e459e8182 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 28 Mar 2019 13:06:03 -0700 Subject: [PATCH 07/13] fix it --- .../org/apache/druid/data/input/orc/OrcStructConverter.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java index e458af640561..e3520819d313 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -49,7 +49,7 @@ public class OrcStructConverter { - static Map convertMap( + private static Map convertMap( TypeDescription fieldDescription, OrcMap map, boolean binaryAsString @@ -154,6 +154,10 @@ Object convertField(OrcStruct struct, String fieldName) TypeDescription fieldDescription = schema.getChildren().get(fieldIndex); WritableComparable fieldValue = struct.getFieldValue(fieldIndex); + if (fieldValue == null) { + return null; + } + if (fieldDescription.getCategory().isPrimitive()) { return convertPrimitive(fieldDescription, fieldValue, binaryAsString); } else { From 6c665ec869b330c17ba2c73645f7dde25e69d0ed Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 29 Mar 2019 18:46:01 -0700 Subject: [PATCH 08/13] exclude redundant dependencies --- .idea/misc.xml | 1 - extensions-core/orc-extensions/pom.xml | 8 ++++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/.idea/misc.xml b/.idea/misc.xml index ada5175311cb..9d0b0220e000 100644 --- a/.idea/misc.xml +++ b/.idea/misc.xml @@ -35,7 +35,6 @@ diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index 42b0bcc7aa0b..13d3c0d409db 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -67,6 +67,14 @@ commons-codec commons-codec + + commons-lang + commons-lang + + + org.slf4j + slf4j-api + From 64ce9e3859da18d200ff468b06076933352d123b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 3 Apr 2019 20:42:05 -0700 Subject: [PATCH 09/13] use latest orc-mapreduce, add hadoop jobProperties recommendations to docs --- .../development/extensions-core/orc.md | 18 ++++++ extensions-core/orc-extensions/pom.xml | 59 ++++++++++++++++++- pom.xml | 1 - 3 files changed, 76 insertions(+), 2 deletions(-) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index 3d7559ca66a8..fed4e8f52cad 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -54,6 +54,24 @@ the set of ingested dimensions, if missing the discovered fields will make up th Auto field discovery will automatically create a string dimension for every (non-timestamp) primitive or `list` of primitives, as well as any flatten expressions defined in the `flattenSpec`. +### Hadoop Job Properties +Like most Hadoop jobs, the best outcomes will add `"mapreduce.job.user.classpath.first": "true"` or +`"mapreduce.job.classloader": "true"` to the `jobProperties` section of `tuningConfig`. Note that it is likely if using +`"mapreduce.job.classloader": "true"` in the `jobProperties` of the indexing task that you will need to set +`mapreduce.job.classloader.system.classes` to include `-org.apache.hadoop.hive.` to instruct Hadoop to load +`org.apache.hadoop.hive` classes from the application jars instead of system jars, e.g. + +```json +... + "mapreduce.job.classloader": "true", + "mapreduce.job.classloader.system.classes" : "java., javax.accessibility., javax.activation., javax.activity., javax.annotation., javax.annotation.processing., javax.crypto., javax.imageio., javax.jws., javax.lang.model., -javax.management.j2ee., javax.management., javax.naming., javax.net., javax.print., javax.rmi., javax.script., -javax.security.auth.message., javax.security.auth., javax.security.cert., javax.security.sasl., javax.sound., javax.sql., javax.swing., javax.tools., javax.transaction., -javax.xml.registry., -javax.xml.rpc., javax.xml., org.w3c.dom., org.xml.sax., org.apache.commons.logging., org.apache.log4j., -org.apache.hadoop.hbase., -org.apache.hadoop.hive., org.apache.hadoop., core-default.xml, hdfs-default.xml, mapred-default.xml, yarn-default.xml", +... +``` + +This is due to the `hive-storage-api` dependency of the +`orc-mapreduce` library, which provides some classes under the `org.apache.hadoop.hive` package. If instead using the +setting `"mapreduce.job.user.classpath.first": "true"`, then this will not be an issue. + ### Examples #### `orc` parser, `orc` parseSpec, auto field discovery, flatten expressions diff --git a/extensions-core/orc-extensions/pom.xml b/extensions-core/orc-extensions/pom.xml index 13d3c0d409db..fa8c9df73421 100644 --- a/extensions-core/orc-extensions/pom.xml +++ b/extensions-core/orc-extensions/pom.xml @@ -33,7 +33,7 @@ 4.0.0 - 1.2.1 + 1.5.5 @@ -63,18 +63,75 @@ com.esotericsoftware kryo-shaded + + com.google.guava + guava + + + com.google.protobuf + protobuf-java + + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + commons-cli + commons-cli + commons-codec commons-codec + + commons-io + commons-io + commons-lang commons-lang + + commons-logging + commons-logging + + + javax.ws.rs + jsr311-api + + + javax.xml.bind + jaxb-api + + + org.apache.hadoop + hadoop-hdfs + + + log4j + log4j + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + org.slf4j slf4j-api + + xmlenc + xmlenc + diff --git a/pom.xml b/pom.xml index 83d0b84cd21c..ab0634239e21 100644 --- a/pom.xml +++ b/pom.xml @@ -94,7 +94,6 @@ 1.7.12 2.8.3 - 2.0.0 1.6.6 1.11.199 2.5.5 From bd8d3af20f46903de6dfc6a5ba8ffb50a0d07b2b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 4 Apr 2019 15:41:56 -0700 Subject: [PATCH 10/13] doc fix --- docs/content/development/extensions-core/orc.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/content/development/extensions-core/orc.md b/docs/content/development/extensions-core/orc.md index fed4e8f52cad..9490ec539ff5 100644 --- a/docs/content/development/extensions-core/orc.md +++ b/docs/content/development/extensions-core/orc.md @@ -57,9 +57,9 @@ primitives, as well as any flatten expressions defined in the `flattenSpec`. ### Hadoop Job Properties Like most Hadoop jobs, the best outcomes will add `"mapreduce.job.user.classpath.first": "true"` or `"mapreduce.job.classloader": "true"` to the `jobProperties` section of `tuningConfig`. Note that it is likely if using -`"mapreduce.job.classloader": "true"` in the `jobProperties` of the indexing task that you will need to set -`mapreduce.job.classloader.system.classes` to include `-org.apache.hadoop.hive.` to instruct Hadoop to load -`org.apache.hadoop.hive` classes from the application jars instead of system jars, e.g. +`"mapreduce.job.classloader": "true"` that you will need to set `mapreduce.job.classloader.system.classes` to include +`-org.apache.hadoop.hive.` to instruct Hadoop to load `org.apache.hadoop.hive` classes from the application jars instead +of system jars, e.g. ```json ... From b53fb3620ac20be9e81f55b145e2c8818b6b27fb Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 8 Apr 2019 15:52:24 -0700 Subject: [PATCH 11/13] review stuff and fix binaryAsString --- .../druid/data/input/orc/OrcHadoopInputRowParser.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java index be179fb49834..53b7088e134f 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -37,7 +37,7 @@ public class OrcHadoopInputRowParser implements InputRowParser { private final ParseSpec parseSpec; - private final ObjectFlattener groupFlattener; + private final ObjectFlattener orcStructFlattener; private final MapInputRowParser parser; private final boolean binaryAsString; @@ -50,12 +50,12 @@ public OrcHadoopInputRowParser( this.parseSpec = parseSpec; this.binaryAsString = binaryAsString == null ? false : binaryAsString; final JSONPathSpec flattenSpec; - if ((parseSpec instanceof OrcParseSpec)) { + if (parseSpec instanceof OrcParseSpec) { flattenSpec = ((OrcParseSpec) parseSpec).getFlattenSpec(); } else { flattenSpec = JSONPathSpec.DEFAULT; } - this.groupFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(false)); + this.orcStructFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(binaryAsString)); this.parser = new MapInputRowParser(parseSpec); } @@ -63,7 +63,7 @@ public OrcHadoopInputRowParser( @Override public List parseBatch(OrcStruct input) { - return parser.parseBatch(groupFlattener.flatten(input)); + return parser.parseBatch(orcStructFlattener.flatten(input)); } @Override From fa83988ce1414704df2f7c4eb2fd40a0a2595dbf Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 8 Apr 2019 17:09:45 -0700 Subject: [PATCH 12/13] cache for root level fields --- .../input/orc/OrcHadoopInputRowParser.java | 2 +- .../data/input/orc/OrcStructConverter.java | 74 +++++++++++++------ .../data/input/orc/OrcStructJsonProvider.java | 5 +- 3 files changed, 57 insertions(+), 24 deletions(-) diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java index 53b7088e134f..38b65d1cee2a 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParser.java @@ -55,7 +55,7 @@ public OrcHadoopInputRowParser( } else { flattenSpec = JSONPathSpec.DEFAULT; } - this.orcStructFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(binaryAsString)); + this.orcStructFlattener = ObjectFlatteners.create(flattenSpec, new OrcStructFlattenerMaker(this.binaryAsString)); this.parser = new MapInputRowParser(parseSpec); } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java index e3520819d313..8931ebb51851 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -19,6 +19,8 @@ package org.apache.druid.data.input.orc; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.hadoop.hive.serde2.io.DateWritable; @@ -49,6 +51,19 @@ public class OrcStructConverter { + @Nonnull + private static List convertList(TypeDescription fieldDescription, OrcList orcList, boolean binaryAsString) + { + // if primitive list, convert primitives + TypeDescription listType = fieldDescription.getChildren().get(0); + if (listType.getCategory().isPrimitive()) { + return (List) orcList.stream() + .map(li -> convertPrimitive(listType, (WritableComparable) li, binaryAsString)) + .collect(Collectors.toList()); + } + return new ArrayList(orcList); + } + private static Map convertMap( TypeDescription fieldDescription, OrcMap map, @@ -126,6 +141,7 @@ private static Object convertPrimitive(TypeDescription fieldDescription, Writabl } private boolean binaryAsString; + private Object2IntMap fieldIndexCache; OrcStructConverter(boolean binaryAsString) { @@ -139,18 +155,46 @@ private static Object convertPrimitive(TypeDescription fieldDescription, Writabl * primitive types will be extracted into an ingestion friendly state (e.g. 'int' and 'long'). Finally, * if a field is not present, this method will return null. * - * Note: "Union" types are not currently supported and will be returned as null + * Note: "Union" types are not currently supported and will be returned as null. Additionally, this method + * has a cache of field names to field index that is ONLY valid for the root level {@link OrcStruct}, and should + * not be used for nested {@link OrcStruct} fields of the root. */ @Nullable Object convertField(OrcStruct struct, String fieldName) { + // this cache is only valid for the root level, to skip the indexOf on fieldNames to get the fieldIndex. TypeDescription schema = struct.getSchema(); - int fieldIndex = schema.getFieldNames().indexOf(fieldName); + final List fields = schema.getFieldNames(); + if (fieldIndexCache == null) { + fieldIndexCache = new Object2IntOpenHashMap<>(fields.size()); + for (int i = 0; i < fields.size(); i++) { + fieldIndexCache.put(fields.get(i), i); + } + } + WritableComparable wc = struct.getFieldValue(fieldName); + + int fieldIndex = fieldIndexCache.getOrDefault(fieldName, -1); + return convertField(struct, fieldIndex); + } + + /** + * Convert a orc struct field as though it were a map, by fieldIndex. Complex types will be transformed + * into java lists and maps when possible ({@link OrcStructConverter#convertList} and + * {@link OrcStructConverter#convertMap}), and + * primitive types will be extracted into an ingestion friendly state (e.g. 'int' and 'long'). Finally, + * if a field is not present, this method will return null. + * + * Note: "Union" types are not currently supported and will be returned as null + */ + @Nullable + Object convertField(OrcStruct struct, int fieldIndex) + { if (fieldIndex < 0) { return null; } + TypeDescription schema = struct.getSchema(); TypeDescription fieldDescription = schema.getChildren().get(fieldIndex); WritableComparable fieldValue = struct.getFieldValue(fieldIndex); @@ -172,13 +216,13 @@ Object convertField(OrcStruct struct, String fieldName) switch (fieldDescription.getCategory()) { case LIST: OrcList orcList = (OrcList) fieldValue; - return convertList(fieldDescription, orcList); + return convertList(fieldDescription, orcList, binaryAsString); case MAP: OrcMap map = (OrcMap) fieldValue; return convertMap(fieldDescription, map, binaryAsString); case STRUCT: OrcStruct structMap = (OrcStruct) fieldValue; - return convertMap(structMap); + return convertStructToMap(structMap); case UNION: // sorry union types :( default: @@ -187,25 +231,13 @@ Object convertField(OrcStruct struct, String fieldName) } } - @Nonnull - private List convertList(TypeDescription fieldDescription, OrcList orcList) - { - // if primitive list, convert primitives - TypeDescription listType = fieldDescription.getChildren().get(0); - if (listType.getCategory().isPrimitive()) { - return (List) orcList.stream() - .map(li -> convertPrimitive(listType, (WritableComparable) li, binaryAsString)) - .collect(Collectors.toList()); - } - return new ArrayList(orcList); - } - - - private Map convertMap(OrcStruct map) + private Map convertStructToMap(OrcStruct map) { Map converted = new HashMap<>(); - for (String key : map.getSchema().getFieldNames()) { - converted.put(key, convertField(map, key)); + List fieldNames = map.getSchema().getFieldNames(); + + for (int i = 0; i < fieldNames.size(); i++) { + converted.put(fieldNames.get(i), convertField(map, i)); } return converted; } diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java index ed74bb5691bc..de84bfbd3ba2 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructJsonProvider.java @@ -106,8 +106,9 @@ public Object getMapValue(final Object o, final String s) } else if (o instanceof Map) { return ((Map) o).get(s); } else if (o instanceof OrcStruct) { - OrcStruct g = (OrcStruct) o; - return converter.convertField(g, s); + OrcStruct struct = (OrcStruct) o; + // get field by index since we have no way to know if this map is the root or not + return converter.convertField(struct, struct.getSchema().getFieldNames().indexOf(s)); } throw new UnsupportedOperationException(o.getClass().getName()); } From f6ea39ba6d0c25cce7a5cad3b845e75565169466 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 8 Apr 2019 19:04:23 -0700 Subject: [PATCH 13/13] more better --- .../druid/data/input/orc/OrcStructConverter.java | 15 +++++---------- .../data/input/orc/OrcStructFlattenerMaker.java | 2 +- 2 files changed, 6 insertions(+), 11 deletions(-) diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java index 8931ebb51851..20fbf0697203 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructConverter.java @@ -149,18 +149,13 @@ private static Object convertPrimitive(TypeDescription fieldDescription, Writabl } /** - * Convert a orc struct field as though it were a map. Complex types will be transformed - * into java lists and maps when possible ({@link OrcStructConverter#convertList} and - * {@link OrcStructConverter#convertMap}), and - * primitive types will be extracted into an ingestion friendly state (e.g. 'int' and 'long'). Finally, - * if a field is not present, this method will return null. - * - * Note: "Union" types are not currently supported and will be returned as null. Additionally, this method - * has a cache of field names to field index that is ONLY valid for the root level {@link OrcStruct}, and should - * not be used for nested {@link OrcStruct} fields of the root. + * Convert a orc struct field of the "root" {@link OrcStruct} that represents the "row". This method has a cache of + * field names to field index that is ONLY valid for this {@link OrcStruct}, and should not be used for + * nested {@link OrcStruct} fields of the row. Looks up field index by field name, and delegates to + * {@link OrcStructConverter#convertField(OrcStruct, int)}. */ @Nullable - Object convertField(OrcStruct struct, String fieldName) + Object convertRootField(OrcStruct struct, String fieldName) { // this cache is only valid for the root level, to skip the indexOf on fieldNames to get the fieldIndex. TypeDescription schema = struct.getSchema(); diff --git a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java index 2171b6c3726a..a7094666255a 100644 --- a/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java +++ b/extensions-core/orc-extensions/src/main/java/org/apache/druid/data/input/orc/OrcStructFlattenerMaker.java @@ -68,7 +68,7 @@ public Iterable discoverRootFields(OrcStruct obj) @Override public Object getRootField(OrcStruct obj, String key) { - return finalizeConversion(converter.convertField(obj, key)); + return finalizeConversion(converter.convertRootField(obj, key)); } @Override