You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2016/09/08 08:45:56 UTC
[1/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Repository: hive
Updated Branches:
refs/heads/master 63fdb513f -> 58d1befa2
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/explain_ddl.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_ddl.q.out b/ql/src/test/results/clientpositive/explain_ddl.q.out
index fa73d99..e8438a1 100644
--- a/ql/src/test/results/clientpositive/explain_ddl.q.out
+++ b/ql/src/test/results/clientpositive/explain_ddl.q.out
@@ -251,6 +251,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Select Operator
expressions: key (type: string), value (type: string)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/explain_logical.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/explain_logical.q.out b/ql/src/test/results/clientpositive/explain_logical.q.out
index cb01724..4a25a38 100644
--- a/ql/src/test/results/clientpositive/explain_logical.q.out
+++ b/ql/src/test/results/clientpositive/explain_logical.q.out
@@ -263,6 +263,8 @@ LOGICAL PLAN:
src
TableScan (TS_0)
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Select Operator (SEL_1)
expressions: key (type: string), value (type: string)
@@ -278,6 +280,8 @@ LOGICAL PLAN:
srcpart
TableScan (TS_0)
alias: srcpart
+ properties:
+ insideView TRUE
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
Select Operator (SEL_2)
expressions: ds (type: string), key (type: string), value (type: string)
@@ -293,6 +297,8 @@ LOGICAL PLAN:
$hdt$_0:srcpart
TableScan (TS_0)
alias: srcpart
+ properties:
+ insideView TRUE
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
Filter Operator (FIL_11)
predicate: key is not null (type: boolean)
@@ -328,6 +334,8 @@ $hdt$_0:srcpart
$hdt$_1:src2
TableScan (TS_3)
alias: src2
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator (FIL_12)
predicate: key is not null (type: boolean)
@@ -359,6 +367,8 @@ LOGICAL PLAN:
$hdt$_0:src
TableScan (TS_0)
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator (FIL_15)
predicate: key is not null (type: boolean)
@@ -396,6 +406,8 @@ $hdt$_0:src
$hdt$_1:srcpart
TableScan (TS_3)
alias: srcpart
+ properties:
+ insideView TRUE
Statistics: Num rows: 2000 Data size: 21248 Basic stats: COMPLETE Column stats: NONE
Filter Operator (FIL_16)
predicate: key is not null (type: boolean)
@@ -423,6 +435,8 @@ $hdt$_1:srcpart
$hdt$_2:src3
TableScan (TS_6)
alias: src3
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator (FIL_17)
predicate: key is not null (type: boolean)
@@ -470,6 +484,8 @@ LOGICAL PLAN:
srcpart
TableScan (TS_0)
alias: srcpart
+ properties:
+ insideView TRUE
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Filter Operator (FIL_4)
predicate: (ds = '10') (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/join_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/join_view.q.out b/ql/src/test/results/clientpositive/join_view.q.out
index 84a5cb8..b1bfb00 100644
--- a/ql/src/test/results/clientpositive/join_view.q.out
+++ b/ql/src/test/results/clientpositive/join_view.q.out
@@ -53,6 +53,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: invites
+ properties:
+ insideView TRUE
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: (ds = '2011-09-01') (type: boolean)
@@ -67,6 +69,8 @@ STAGE PLANS:
value expressions: _col0 (type: string)
TableScan
alias: invites2
+ properties:
+ insideView TRUE
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: (ds = '2011-09-01') (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
index be09d28..ef01018 100644
--- a/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/explainuser_1.q.out
@@ -1951,7 +1951,7 @@ Stage-0
Filter Operator [FIL_15] (rows=166 width=178)
predicate:((value > 'val_9') and key is not null)
TableScan [TS_0] (rows=500 width=178)
- default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"]
+ default@src_cbo,b,Tbl:COMPLETE,Col:COMPLETE,Output:["key","value"],properties:{"insideView":"TRUE"}
<-Map 3 [SIMPLE_EDGE] llap
SHUFFLE [RS_9]
PartitionCols:_col0, _col1
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_2.q.out b/ql/src/test/results/clientpositive/masking_2.q.out
index ff045a9..4900a5a 100644
--- a/ql/src/test/results/clientpositive/masking_2.q.out
+++ b/ql/src/test/results/clientpositive/masking_2.q.out
@@ -22,6 +22,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -74,6 +76,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -139,6 +143,8 @@ STAGE PLANS:
value expressions: _col1 (type: string)
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0) and reverse(value) is not null) (type: boolean)
@@ -190,6 +196,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -206,6 +214,8 @@ STAGE PLANS:
value expressions: _col0 (type: int), _col1 (type: string)
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0) and reverse(value) is not null) (type: boolean)
@@ -257,6 +267,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -279,6 +291,8 @@ STAGE PLANS:
Statistics: Num rows: 110 Data size: 1167 Basic stats: COMPLETE Column stats: NONE
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_6.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_6.q.out b/ql/src/test/results/clientpositive/masking_6.q.out
index fb8c90f..56e4006 100644
--- a/ql/src/test/results/clientpositive/masking_6.q.out
+++ b/ql/src/test/results/clientpositive/masking_6.q.out
@@ -26,6 +26,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -78,6 +80,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -149,6 +153,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -201,6 +207,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/masking_7.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/masking_7.q.out b/ql/src/test/results/clientpositive/masking_7.q.out
index 1fa9311..c4531f8 100644
--- a/ql/src/test/results/clientpositive/masking_7.q.out
+++ b/ql/src/test/results/clientpositive/masking_7.q.out
@@ -26,6 +26,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -78,6 +80,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
@@ -149,6 +153,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10)) (type: boolean)
@@ -201,6 +207,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (((UDFToInteger(key) % 2) = 0) and (UDFToInteger(key) < 10) and (UDFToInteger(key) > 0)) (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/serde_user_properties.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/serde_user_properties.q.out b/ql/src/test/results/clientpositive/serde_user_properties.q.out
index d5b81ed..c671d33 100644
--- a/ql/src/test/results/clientpositive/serde_user_properties.q.out
+++ b/ql/src/test/results/clientpositive/serde_user_properties.q.out
@@ -184,6 +184,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ user.defined.key some.value
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Select Operator
@@ -310,6 +312,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: a
+ properties:
+ user.defined.key some.value
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
GatherStats: false
Select Operator
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/show_functions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_functions.q.out b/ql/src/test/results/clientpositive/show_functions.q.out
index 953c98a..18c7b0e 100644
--- a/ql/src/test/results/clientpositive/show_functions.q.out
+++ b/ql/src/test/results/clientpositive/show_functions.q.out
@@ -87,6 +87,14 @@ field
find_in_set
first_value
floor
+floor_day
+floor_hour
+floor_minute
+floor_month
+floor_quarter
+floor_second
+floor_week
+floor_year
format_number
from_unixtime
from_utc_timestamp
@@ -286,6 +294,7 @@ e
encode
explode
first_value
+floor_minute
from_unixtime
in_file
inline
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/spark/join_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/spark/join_view.q.out b/ql/src/test/results/clientpositive/spark/join_view.q.out
index 8a8aa0a..fc78988 100644
--- a/ql/src/test/results/clientpositive/spark/join_view.q.out
+++ b/ql/src/test/results/clientpositive/spark/join_view.q.out
@@ -58,6 +58,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: invites
+ properties:
+ insideView TRUE
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: (ds = '2011-09-01') (type: boolean)
@@ -74,6 +76,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: invites2
+ properties:
+ insideView TRUE
Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
Filter Operator
predicate: (ds = '2011-09-01') (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/subquery_notin.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_notin.q.out b/ql/src/test/results/clientpositive/subquery_notin.q.out
index 1c6b545..c184806 100644
--- a/ql/src/test/results/clientpositive/subquery_notin.q.out
+++ b/ql/src/test/results/clientpositive/subquery_notin.q.out
@@ -1473,6 +1473,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (true) ELSE (key is null) END) (type: boolean)
@@ -1511,6 +1513,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (key < '11') (type: boolean)
@@ -1554,6 +1558,8 @@ STAGE PLANS:
Statistics: Num rows: 166 Data size: 3257 Basic stats: COMPLETE Column stats: NONE
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: ((key < '11') and CASE WHEN ((key > '104')) THEN (null) ELSE ((key < '11')) END) (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/subquery_views.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/subquery_views.q.out b/ql/src/test/results/clientpositive/subquery_views.q.out
index 046f0fe..610bf24 100644
--- a/ql/src/test/results/clientpositive/subquery_views.q.out
+++ b/ql/src/test/results/clientpositive/subquery_views.q.out
@@ -175,6 +175,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: b
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (key < '11') (type: boolean)
@@ -330,6 +332,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: b
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (key < '11') (type: boolean)
[5/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
new file mode 100644
index 0000000..0b87976
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTopNQueryRecordReader.java
@@ -0,0 +1,106 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Iterators;
+
+import io.druid.query.Result;
+import io.druid.query.topn.DimensionAndMetricValueExtractor;
+import io.druid.query.topn.TopNQuery;
+import io.druid.query.topn.TopNResultValue;
+
+/**
+ * Record reader for results for Druid TopNQuery.
+ */
+public class DruidTopNQueryRecordReader
+ extends DruidQueryRecordReader<TopNQuery, Result<TopNResultValue>> {
+
+ private Result<TopNResultValue> current;
+ private Iterator<DimensionAndMetricValueExtractor> values = Iterators.emptyIterator();
+
+ @Override
+ protected TopNQuery createQuery(String content) throws IOException {
+ return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, TopNQuery.class);
+ }
+
+ @Override
+ protected List<Result<TopNResultValue>> createResultsList(InputStream content) throws IOException {
+ return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+ new TypeReference<List<Result<TopNResultValue>>>(){});
+ }
+
+ @Override
+ public boolean nextKeyValue() {
+ if (values.hasNext()) {
+ return true;
+ }
+ if (results.hasNext()) {
+ current = results.next();
+ values = current.getValue().getValue().iterator();
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public NullWritable getCurrentKey() throws IOException, InterruptedException {
+ return NullWritable.get();
+ }
+
+ @Override
+ public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+ // Create new value
+ DruidWritable value = new DruidWritable();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ if (values.hasNext()) {
+ value.getValue().putAll(values.next().getBaseObject());
+ return value;
+ }
+ return value;
+ }
+
+ @Override
+ public boolean next(NullWritable key, DruidWritable value) {
+ if (nextKeyValue()) {
+ // Update value
+ value.getValue().clear();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ if (values.hasNext()) {
+ value.getValue().putAll(values.next().getBaseObject());
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public float getProgress() {
+ return results.hasNext() || values.hasNext() ? 0 : 1;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
new file mode 100644
index 0000000..77ffcd4
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidWritable.java
@@ -0,0 +1,81 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.io.Writable;
+
+import com.google.common.base.Objects;
+
+/**
+ * Writable for Druid results.
+ */
+public class DruidWritable implements Writable {
+
+ private final Map<String, Object> value;
+
+ public DruidWritable() {
+ value = new HashMap<>();
+ }
+
+ public DruidWritable(Map<String, Object> value) {
+ this.value = value;
+ }
+
+ public Map<String, Object> getValue() {
+ return value;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hashCode(value);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ return Objects.equal(value, ((DruidWritable) o).value);
+ }
+
+ @Override
+ public String toString() {
+ return "DruidWritable{value=" + value + '}';
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
new file mode 100644
index 0000000..2b4df78
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidSerDe.java
@@ -0,0 +1,88 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.util.StringUtils;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+
+/**
+ * Druid SerDe to be used in tests.
+ */
+public class QTestDruidSerDe extends DruidSerDe {
+
+ // Request :
+ // "{\"queryType\":\"segmentMetadata\",\"dataSource\":{\"type\":\"table\",\"name\":\"wikipedia\"},"
+ // + "\"intervals\":{\"type\":\"intervals\","
+ // + "\"intervals\":[\"-146136543-09-08T00:30:34.096-07:52:58/146140482-04-24T08:36:27.903-07:00\"]},"
+ // + "\"toInclude\":{\"type\":\"all\"},\"merge\":true,\"context\":null,\"analysisTypes\":[],"
+ // + "\"usingDefaultInterval\":true,\"lenientAggregatorMerge\":false,\"descending\":false}";
+ private static final String RESPONSE =
+ "[ {\r\n "
+ + " \"id\" : \"merged\",\r\n "
+ + " \"intervals\" : [ \"2010-01-01T00:00:00.000Z/2015-12-31T00:00:00.000Z\" ],\r\n "
+ + " \"columns\" : {\r\n "
+ + " \"__time\" : { \"type\" : \"LONG\", \"hasMultipleValues\" : false, \"size\" : 407240380, \"cardinality\" : null, \"errorMessage\" : null },\r\n "
+ + " \"robot\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"namespace\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : true, \"size\" : 100000, \"cardinality\" : 1504, \"errorMessage\" : null },\r\n "
+ + " \"anonymous\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"unpatrolled\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"page\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"language\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"newpage\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"user\" : { \"type\" : \"STRING\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : 1944, \"errorMessage\" : null },\r\n "
+ + " \"count\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n "
+ + " \"added\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n "
+ + " \"delta\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n "
+ + " \"variation\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null },\r\n "
+ + " \"deleted\" : { \"type\" : \"FLOAT\", \"hasMultipleValues\" : false, \"size\" : 100000, \"cardinality\" : null, \"errorMessage\" : null }\r\n "
+ + " },\r\n "
+ + " \"aggregators\" : {\r\n "
+ + " \"count\" : { \"type\" : \"longSum\", \"name\" : \"count\", \"fieldName\" : \"count\" },\r\n "
+ + " \"added\" : { \"type\" : \"doubleSum\", \"name\" : \"added\", \"fieldName\" : \"added\" },\r\n "
+ + " \"delta\" : { \"type\" : \"doubleSum\", \"name\" : \"delta\", \"fieldName\" : \"delta\" },\r\n "
+ + " \"variation\" : { \"type\" : \"doubleSum\", \"name\" : \"variation\", \"fieldName\" : \"variation\" },\r\n "
+ + " \"deleted\" : { \"type\" : \"doubleSum\", \"name\" : \"deleted\", \"fieldName\" : \"deleted\" }\r\n "
+ + " },\r\n "
+ + " \"queryGranularity\" : {\r\n \"type\": \"none\"\r\n },\r\n "
+ + " \"size\" : 300000,\r\n "
+ + " \"numRows\" : 5000000\r\n} ]";
+
+ /* Submits the request and returns */
+ @Override
+ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query)
+ throws SerDeException {
+ // Retrieve results
+ List<SegmentAnalysis> resultsList;
+ try {
+ resultsList = DruidStorageHandlerUtils.JSON_MAPPER.readValue(RESPONSE,
+ new TypeReference<List<SegmentAnalysis>>() {});
+ } catch (Exception e) {
+ throw new SerDeException(StringUtils.stringifyException(e));
+ }
+ return resultsList.get(0);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
new file mode 100644
index 0000000..0a44aaa
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/QTestDruidStorageHandler.java
@@ -0,0 +1,34 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import org.apache.hadoop.hive.serde2.SerDe;
+
+/**
+ * Storage handler for Druid to be used in tests. It cannot connect to
+ * Druid, and thus it cannot execute queries.
+ */
+@SuppressWarnings("deprecation")
+public class QTestDruidStorageHandler extends DruidStorageHandler {
+
+ @Override
+ public Class<? extends SerDe> getSerDeClass() {
+ return QTestDruidSerDe.class;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
new file mode 100644
index 0000000..9c5c65c
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestDruidSerDe.java
@@ -0,0 +1,576 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.sql.Timestamp;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidWritable;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeUtils;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import io.druid.data.input.Row;
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.query.Query;
+import io.druid.query.Result;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.select.SelectResultValue;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.timeseries.TimeseriesResultValue;
+import io.druid.query.topn.TopNQuery;
+import io.druid.query.topn.TopNResultValue;
+
+/**
+ * Basic tests for Druid SerDe. The examples are taken from Druid 0.9.1.1
+ * documentation.
+ */
+public class TestDruidSerDe {
+
+ // Timeseries query
+ private static final String TIMESERIES_QUERY =
+ "{ \"queryType\": \"timeseries\", "
+ + " \"dataSource\": \"sample_datasource\", "
+ + " \"granularity\": \"day\", "
+ + " \"descending\": \"true\", "
+ + " \"filter\": { "
+ + " \"type\": \"and\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"selector\", \"dimension\": \"sample_dimension1\", \"value\": \"sample_value1\" }, "
+ + " { \"type\": \"or\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"selector\", \"dimension\": \"sample_dimension2\", \"value\": \"sample_value2\" }, "
+ + " { \"type\": \"selector\", \"dimension\": \"sample_dimension3\", \"value\": \"sample_value3\" } "
+ + " ] "
+ + " } "
+ + " ] "
+ + " }, "
+ + " \"aggregations\": [ "
+ + " { \"type\": \"longSum\", \"name\": \"sample_name1\", \"fieldName\": \"sample_fieldName1\" }, "
+ + " { \"type\": \"doubleSum\", \"name\": \"sample_name2\", \"fieldName\": \"sample_fieldName2\" } "
+ + " ], "
+ + " \"postAggregations\": [ "
+ + " { \"type\": \"arithmetic\", "
+ + " \"name\": \"sample_divide\", "
+ + " \"fn\": \"/\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name1\", \"fieldName\": \"sample_name1\" }, "
+ + " { \"type\": \"fieldAccess\", \"name\": \"postAgg__sample_name2\", \"fieldName\": \"sample_name2\" } "
+ + " ] "
+ + " } "
+ + " ], "
+ + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ]}";
+ // Timeseries query results
+ private static final String TIMESERIES_QUERY_RESULTS =
+ "[ "
+ + "{ "
+ + " \"timestamp\": \"2012-01-01T00:00:00.000Z\", "
+ + " \"result\": { \"sample_name1\": 0, \"sample_name2\": 1.0, \"sample_divide\": 2.2222 } "
+ + "}, "
+ + "{ "
+ + " \"timestamp\": \"2012-01-02T00:00:00.000Z\", "
+ + " \"result\": { \"sample_name1\": 2, \"sample_name2\": 3.32, \"sample_divide\": 4 } "
+ + "}]";
+ // Timeseries query results as records
+ private static final Object[][] TIMESERIES_QUERY_RESULTS_RECORDS = new Object[][] {
+ new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new LongWritable(0), new FloatWritable(1.0F), new FloatWritable(2.2222F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1325462400000L)), new LongWritable(2), new FloatWritable(3.32F), new FloatWritable(4F)}
+ };
+
+ // TopN query
+ private static final String TOPN_QUERY =
+ "{ \"queryType\": \"topN\", "
+ + " \"dataSource\": \"sample_data\", "
+ + " \"dimension\": \"sample_dim\", "
+ + " \"threshold\": 5, "
+ + " \"metric\": \"count\", "
+ + " \"granularity\": \"all\", "
+ + " \"filter\": { "
+ + " \"type\": \"and\", "
+ + " \"fields\": [ "
+ + " { "
+ + " \"type\": \"selector\", "
+ + " \"dimension\": \"dim1\", "
+ + " \"value\": \"some_value\" "
+ + " }, "
+ + " { "
+ + " \"type\": \"selector\", "
+ + " \"dimension\": \"dim2\", "
+ + " \"value\": \"some_other_val\" "
+ + " } "
+ + " ] "
+ + " }, "
+ + " \"aggregations\": [ "
+ + " { "
+ + " \"type\": \"longSum\", "
+ + " \"name\": \"count\", "
+ + " \"fieldName\": \"count\" "
+ + " }, "
+ + " { "
+ + " \"type\": \"doubleSum\", "
+ + " \"name\": \"some_metric\", "
+ + " \"fieldName\": \"some_metric\" "
+ + " } "
+ + " ], "
+ + " \"postAggregations\": [ "
+ + " { "
+ + " \"type\": \"arithmetic\", "
+ + " \"name\": \"sample_divide\", "
+ + " \"fn\": \"/\", "
+ + " \"fields\": [ "
+ + " { "
+ + " \"type\": \"fieldAccess\", "
+ + " \"name\": \"some_metric\", "
+ + " \"fieldName\": \"some_metric\" "
+ + " }, "
+ + " { "
+ + " \"type\": \"fieldAccess\", "
+ + " \"name\": \"count\", "
+ + " \"fieldName\": \"count\" "
+ + " } "
+ + " ] "
+ + " } "
+ + " ], "
+ + " \"intervals\": [ "
+ + " \"2013-08-31T00:00:00.000/2013-09-03T00:00:00.000\" "
+ + " ]}";
+ // TopN query results
+ private static final String TOPN_QUERY_RESULTS =
+ "[ "
+ + " { "
+ + " \"timestamp\": \"2013-08-31T00:00:00.000Z\", "
+ + " \"result\": [ "
+ + " { "
+ + " \"sample_dim\": \"dim1_val\", "
+ + " \"count\": 111, "
+ + " \"some_metric\": 10669, "
+ + " \"sample_divide\": 96.11711711711712 "
+ + " }, "
+ + " { "
+ + " \"sample_dim\": \"another_dim1_val\", "
+ + " \"count\": 88, "
+ + " \"some_metric\": 28344, "
+ + " \"sample_divide\": 322.09090909090907 "
+ + " }, "
+ + " { "
+ + " \"sample_dim\": \"dim1_val3\", "
+ + " \"count\": 70, "
+ + " \"some_metric\": 871, "
+ + " \"sample_divide\": 12.442857142857143 "
+ + " }, "
+ + " { "
+ + " \"sample_dim\": \"dim1_val4\", "
+ + " \"count\": 62, "
+ + " \"some_metric\": 815, "
+ + " \"sample_divide\": 13.14516129032258 "
+ + " }, "
+ + " { "
+ + " \"sample_dim\": \"dim1_val5\", "
+ + " \"count\": 60, "
+ + " \"some_metric\": 2787, "
+ + " \"sample_divide\": 46.45 "
+ + " } "
+ + " ] "
+ + " }]";
+ // TopN query results as records
+ private static final Object[][] TOPN_QUERY_RESULTS_RECORDS = new Object[][] {
+ new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val"), new LongWritable(111), new FloatWritable(10669F), new FloatWritable(96.11711711711712F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("another_dim1_val"), new LongWritable(88), new FloatWritable(28344F), new FloatWritable(322.09090909090907F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val3"), new LongWritable(70), new FloatWritable(871F), new FloatWritable(12.442857142857143F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val4"), new LongWritable(62), new FloatWritable(815F), new FloatWritable(13.14516129032258F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1377907200000L)), new Text("dim1_val5"), new LongWritable(60), new FloatWritable(2787F), new FloatWritable(46.45F) }
+ };
+
+ // GroupBy query
+ private static final String GROUP_BY_QUERY =
+ "{ "
+ + " \"queryType\": \"groupBy\", "
+ + " \"dataSource\": \"sample_datasource\", "
+ + " \"granularity\": \"day\", "
+ + " \"dimensions\": [\"country\", \"device\"], "
+ + " \"limitSpec\": {"
+ + " \"type\": \"default\","
+ + " \"limit\": 5000,"
+ + " \"columns\": [\"country\", \"data_transfer\"] }, "
+ + " \"filter\": { "
+ + " \"type\": \"and\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"selector\", \"dimension\": \"carrier\", \"value\": \"AT&T\" }, "
+ + " { \"type\": \"or\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Apple\" }, "
+ + " { \"type\": \"selector\", \"dimension\": \"make\", \"value\": \"Samsung\" } "
+ + " ] "
+ + " } "
+ + " ] "
+ + " }, "
+ + " \"aggregations\": [ "
+ + " { \"type\": \"longSum\", \"name\": \"total_usage\", \"fieldName\": \"user_count\" }, "
+ + " { \"type\": \"doubleSum\", \"name\": \"data_transfer\", \"fieldName\": \"data_transfer\" } "
+ + " ], "
+ + " \"postAggregations\": [ "
+ + " { \"type\": \"arithmetic\", "
+ + " \"name\": \"avg_usage\", "
+ + " \"fn\": \"/\", "
+ + " \"fields\": [ "
+ + " { \"type\": \"fieldAccess\", \"fieldName\": \"data_transfer\" }, "
+ + " { \"type\": \"fieldAccess\", \"fieldName\": \"total_usage\" } "
+ + " ] "
+ + " } "
+ + " ], "
+ + " \"intervals\": [ \"2012-01-01T00:00:00.000/2012-01-03T00:00:00.000\" ], "
+ + " \"having\": { "
+ + " \"type\": \"greaterThan\", "
+ + " \"aggregation\": \"total_usage\", "
+ + " \"value\": 100 "
+ + " }}";
+ // GroupBy query results
+ private static final String GROUP_BY_QUERY_RESULTS =
+ "[ "
+ + " { "
+ + " \"version\" : \"v1\", "
+ + " \"timestamp\" : \"2012-01-01T00:00:00.000Z\", "
+ + " \"event\" : { "
+ + " \"country\" : \"India\", "
+ + " \"device\" : \"phone\", "
+ + " \"total_usage\" : 88, "
+ + " \"data_transfer\" : 29.91233453, "
+ + " \"avg_usage\" : 60.32 "
+ + " } "
+ + " }, "
+ + " { "
+ + " \"version\" : \"v1\", "
+ + " \"timestamp\" : \"2012-01-01T00:00:12.000Z\", "
+ + " \"event\" : { "
+ + " \"country\" : \"Spain\", "
+ + " \"device\" : \"pc\", "
+ + " \"total_usage\" : 16, "
+ + " \"data_transfer\" : 172.93494959, "
+ + " \"avg_usage\" : 6.333333 "
+ + " } "
+ + " }]";
+ // GroupBy query results as records
+ private static final Object[][] GROUP_BY_QUERY_RESULTS_RECORDS = new Object[][] {
+ new Object[] { new TimestampWritable(new Timestamp(1325376000000L)), new Text("India"), new Text("phone"), new LongWritable(88), new FloatWritable(29.91233453F), new FloatWritable(60.32F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1325376012000L)), new Text("Spain"), new Text("pc"), new LongWritable(16), new FloatWritable(172.93494959F), new FloatWritable(6.333333F) }
+ };
+
+ // Select query
+ private static final String SELECT_QUERY =
+ "{ \"queryType\": \"select\", "
+ + " \"dataSource\": \"wikipedia\", \"descending\": \"false\", "
+ + " \"dimensions\":[\"robot\",\"namespace\",\"anonymous\",\"unpatrolled\",\"page\",\"language\",\"newpage\",\"user\"], "
+ + " \"metrics\":[\"count\",\"added\",\"delta\",\"variation\",\"deleted\"], "
+ + " \"granularity\": \"all\", "
+ + " \"intervals\": [ \"2013-01-01/2013-01-02\" ], "
+ + " \"pagingSpec\":{\"pagingIdentifiers\": {}, \"threshold\":5} }";
+ // Select query results
+ private static final String SELECT_QUERY_RESULTS =
+ "[{ "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"result\" : { "
+ + " \"pagingIdentifiers\" : { "
+ + " \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\" : 4 }, "
+ + " \"events\" : [ { "
+ + " \"segmentId\" : \"wikipedia_editstream_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", "
+ + " \"offset\" : 0, "
+ + " \"event\" : { "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"robot\" : \"1\", "
+ + " \"namespace\" : \"article\", "
+ + " \"anonymous\" : \"0\", "
+ + " \"unpatrolled\" : \"0\", "
+ + " \"page\" : \"11._korpus_(NOVJ)\", "
+ + " \"language\" : \"sl\", "
+ + " \"newpage\" : \"0\", "
+ + " \"user\" : \"EmausBot\", "
+ + " \"count\" : 1.0, "
+ + " \"added\" : 39.0, "
+ + " \"delta\" : 39.0, "
+ + " \"variation\" : 39.0, "
+ + " \"deleted\" : 0.0 "
+ + " } "
+ + " }, { "
+ + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", "
+ + " \"offset\" : 1, "
+ + " \"event\" : { "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"robot\" : \"0\", "
+ + " \"namespace\" : \"article\", "
+ + " \"anonymous\" : \"0\", "
+ + " \"unpatrolled\" : \"0\", "
+ + " \"page\" : \"112_U.S._580\", "
+ + " \"language\" : \"en\", "
+ + " \"newpage\" : \"1\", "
+ + " \"user\" : \"MZMcBride\", "
+ + " \"count\" : 1.0, "
+ + " \"added\" : 70.0, "
+ + " \"delta\" : 70.0, "
+ + " \"variation\" : 70.0, "
+ + " \"deleted\" : 0.0 "
+ + " } "
+ + " }, { "
+ + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", "
+ + " \"offset\" : 2, "
+ + " \"event\" : { "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"robot\" : \"0\", "
+ + " \"namespace\" : \"article\", "
+ + " \"anonymous\" : \"0\", "
+ + " \"unpatrolled\" : \"0\", "
+ + " \"page\" : \"113_U.S._243\", "
+ + " \"language\" : \"en\", "
+ + " \"newpage\" : \"1\", "
+ + " \"user\" : \"MZMcBride\", "
+ + " \"count\" : 1.0, "
+ + " \"added\" : 77.0, "
+ + " \"delta\" : 77.0, "
+ + " \"variation\" : 77.0, "
+ + " \"deleted\" : 0.0 "
+ + " } "
+ + " }, { "
+ + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", "
+ + " \"offset\" : 3, "
+ + " \"event\" : { "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"robot\" : \"0\", "
+ + " \"namespace\" : \"article\", "
+ + " \"anonymous\" : \"0\", "
+ + " \"unpatrolled\" : \"0\", "
+ + " \"page\" : \"113_U.S._73\", "
+ + " \"language\" : \"en\", "
+ + " \"newpage\" : \"1\", "
+ + " \"user\" : \"MZMcBride\", "
+ + " \"count\" : 1.0, "
+ + " \"added\" : 70.0, "
+ + " \"delta\" : 70.0, "
+ + " \"variation\" : 70.0, "
+ + " \"deleted\" : 0.0 "
+ + " } "
+ + " }, { "
+ + " \"segmentId\" : \"wikipedia_2012-12-29T00:00:00.000Z_2013-01-10T08:00:00.000Z_2013-01-10T08:13:47.830Z_v9\", "
+ + " \"offset\" : 4, "
+ + " \"event\" : { "
+ + " \"timestamp\" : \"2013-01-01T00:00:00.000Z\", "
+ + " \"robot\" : \"0\", "
+ + " \"namespace\" : \"article\", "
+ + " \"anonymous\" : \"0\", "
+ + " \"unpatrolled\" : \"0\", "
+ + " \"page\" : \"113_U.S._756\", "
+ + " \"language\" : \"en\", "
+ + " \"newpage\" : \"1\", "
+ + " \"user\" : \"MZMcBride\", "
+ + " \"count\" : 1.0, "
+ + " \"added\" : 68.0, "
+ + " \"delta\" : 68.0, "
+ + " \"variation\" : 68.0, "
+ + " \"deleted\" : 0.0 "
+ + " } "
+ + " } ] }} ]";
+ // Select query results as records
+ private static final Object[][] SELECT_QUERY_RESULTS_RECORDS = new Object[][] {
+ new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("1"), new Text("article"), new Text("0"), new Text("0"),
+ new Text("11._korpus_(NOVJ)"), new Text("sl"), new Text("0"), new Text("EmausBot"),
+ new FloatWritable(1.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(39.0F), new FloatWritable(0.0F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+ new Text("112_U.S._580"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+ new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+ new Text("113_U.S._243"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+ new FloatWritable(1.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(77.0F), new FloatWritable(0.0F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+ new Text("113_U.S._73"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+ new FloatWritable(1.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(70.0F), new FloatWritable(0.0F) } ,
+ new Object[] { new TimestampWritable(new Timestamp(1356998400000L)), new Text("0"), new Text("article"), new Text("0"), new Text("0"),
+ new Text("113_U.S._756"), new Text("en"), new Text("1"), new Text("MZMcBride"),
+ new FloatWritable(1.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(68.0F), new FloatWritable(0.0F) }
+ };
+
+
+ /**
+ * Test the default behavior of the objects and object inspectors.
+ * @throws IOException
+ * @throws IllegalAccessException
+ * @throws IllegalArgumentException
+ * @throws SecurityException
+ * @throws NoSuchFieldException
+ * @throws JsonMappingException
+ * @throws JsonParseException
+ * @throws InvocationTargetException
+ * @throws NoSuchMethodException
+ */
+ @Test
+ public void testDruidSerDe()
+ throws SerDeException, JsonParseException, JsonMappingException,
+ NoSuchFieldException, SecurityException, IllegalArgumentException,
+ IllegalAccessException, IOException, InterruptedException,
+ NoSuchMethodException, InvocationTargetException {
+ // Create, initialize, and test the SerDe
+ DruidSerDe serDe = new DruidSerDe();
+ Configuration conf = new Configuration();
+ Properties tbl;
+ // Timeseries query
+ tbl = createPropertiesQuery("sample_datasource", Query.TIMESERIES, TIMESERIES_QUERY);
+ SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+ deserializeQueryResults(serDe, Query.TIMESERIES, TIMESERIES_QUERY,
+ TIMESERIES_QUERY_RESULTS, TIMESERIES_QUERY_RESULTS_RECORDS);
+ // TopN query
+ tbl = createPropertiesQuery("sample_data", Query.TOPN, TOPN_QUERY);
+ SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+ deserializeQueryResults(serDe, Query.TOPN, TOPN_QUERY,
+ TOPN_QUERY_RESULTS, TOPN_QUERY_RESULTS_RECORDS);
+ // GroupBy query
+ tbl = createPropertiesQuery("sample_datasource", Query.GROUP_BY, GROUP_BY_QUERY);
+ SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+ deserializeQueryResults(serDe, Query.GROUP_BY, GROUP_BY_QUERY,
+ GROUP_BY_QUERY_RESULTS, GROUP_BY_QUERY_RESULTS_RECORDS);
+ // Select query
+ tbl = createPropertiesQuery("wikipedia", Query.SELECT, SELECT_QUERY);
+ SerDeUtils.initializeSerDe(serDe, conf, tbl, null);
+ deserializeQueryResults(serDe, Query.SELECT, SELECT_QUERY,
+ SELECT_QUERY_RESULTS, SELECT_QUERY_RESULTS_RECORDS);
+ }
+
+ private static Properties createPropertiesQuery(String dataSource, String queryType, String jsonQuery) {
+ Properties tbl = new Properties();
+
+ // Set the configuration parameters
+ tbl.setProperty(Constants.DRUID_DATA_SOURCE, dataSource);
+ tbl.setProperty(Constants.DRUID_QUERY_JSON, jsonQuery);
+ tbl.setProperty(Constants.DRUID_QUERY_TYPE, queryType);
+ return tbl;
+ }
+
+ private static void deserializeQueryResults(DruidSerDe serDe, String queryType, String jsonQuery,
+ String resultString, Object[][] records) throws SerDeException, JsonParseException,
+ JsonMappingException, IOException, NoSuchFieldException, SecurityException,
+ IllegalArgumentException, IllegalAccessException, InterruptedException,
+ NoSuchMethodException, InvocationTargetException {
+
+ // Initialize
+ Query<?> query = null;
+ DruidQueryRecordReader<?,?> reader = null;
+ List<?> resultsList = null;
+ ObjectMapper mapper = new DefaultObjectMapper();
+ switch (queryType) {
+ case Query.TIMESERIES:
+ query = mapper.readValue(jsonQuery, TimeseriesQuery.class);
+ reader = new DruidTimeseriesQueryRecordReader();
+ resultsList = mapper.readValue(resultString,
+ new TypeReference<List<Result<TimeseriesResultValue>>>() {});
+ break;
+ case Query.TOPN:
+ query = mapper.readValue(jsonQuery, TopNQuery.class);
+ reader = new DruidTopNQueryRecordReader();
+ resultsList = mapper.readValue(resultString,
+ new TypeReference<List<Result<TopNResultValue>>>() {});
+ break;
+ case Query.GROUP_BY:
+ query = mapper.readValue(jsonQuery, GroupByQuery.class);
+ reader = new DruidGroupByQueryRecordReader();
+ resultsList = mapper.readValue(resultString,
+ new TypeReference<List<Row>>() {});
+ break;
+ case Query.SELECT:
+ query = mapper.readValue(jsonQuery, SelectQuery.class);
+ reader = new DruidSelectQueryRecordReader();
+ resultsList = mapper.readValue(resultString,
+ new TypeReference<List<Result<SelectResultValue>>>() {});
+ break;
+ }
+
+ // Set query and fields access
+ Field field1 = DruidQueryRecordReader.class.getDeclaredField("query");
+ field1.setAccessible(true);
+ field1.set(reader, query);
+ if (reader instanceof DruidGroupByQueryRecordReader) {
+ Method method1 = DruidGroupByQueryRecordReader.class.getDeclaredMethod("initExtractors");
+ method1.setAccessible(true);
+ method1.invoke(reader);
+ }
+ Field field2 = DruidQueryRecordReader.class.getDeclaredField("results");
+ field2.setAccessible(true);
+
+ // Get the row structure
+ StructObjectInspector oi = (StructObjectInspector) serDe.getObjectInspector();
+ List<? extends StructField> fieldRefs = oi.getAllStructFieldRefs();
+
+ // Check mapred
+ Iterator<?> results = resultsList.iterator();
+ field2.set(reader, results);
+ DruidWritable writable = new DruidWritable();
+ int pos = 0;
+ while (reader.next(NullWritable.get(), writable)) {
+ Object row = serDe.deserialize(writable);
+ Object[] expectedFieldsData = records[pos];
+ assertEquals(expectedFieldsData.length, fieldRefs.size());
+ for (int i = 0; i < fieldRefs.size(); i++) {
+ Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
+ assertEquals("Field " + i, expectedFieldsData[i], fieldData);
+ }
+ pos++;
+ }
+ assertEquals(pos, records.length);
+
+ // Check mapreduce
+ results = resultsList.iterator();
+ field2.set(reader, results);
+ pos = 0;
+ while (reader.nextKeyValue()) {
+ Object row = serDe.deserialize(reader.getCurrentValue());
+ Object[] expectedFieldsData = records[pos];
+ assertEquals(expectedFieldsData.length, fieldRefs.size());
+ for (int i = 0; i < fieldRefs.size(); i++) {
+ Object fieldData = oi.getStructFieldData(row, fieldRefs.get(i));
+ assertEquals("Field " + i, expectedFieldsData[i], fieldData);
+ }
+ pos++;
+ }
+ assertEquals(pos, records.length);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
new file mode 100644
index 0000000..b20168d
--- /dev/null
+++ b/druid-handler/src/test/org/apache/hadoop/hive/druid/TestHiveDruidQueryBasedInputFormat.java
@@ -0,0 +1,101 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.joda.time.Interval;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestHiveDruidQueryBasedInputFormat extends TestCase {
+
+ @SuppressWarnings("unchecked")
+ @Test
+ public void testCreateSplitsIntervals() throws Exception {
+ HiveDruidQueryBasedInputFormat input = new HiveDruidQueryBasedInputFormat();
+
+ Method method1 = HiveDruidQueryBasedInputFormat.class.getDeclaredMethod("createSplitsIntervals",
+ List.class, int.class);
+ method1.setAccessible(true);
+
+ List<Interval> intervals;
+ List<List<Interval>> resultList;
+ List<List<Interval>> expectedResultList;
+
+ // Test 1 : single split, create 4
+ intervals = new ArrayList<>();
+ intervals.add(new Interval(1262304000000L, 1293840000000L));
+ resultList = (List<List<Interval>>) method1.invoke(input, intervals, 4);
+ expectedResultList = new ArrayList<>();
+ expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1270188000000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1270188000000L, 1278072000000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1278072000000L, 1285956000000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1285956000000L, 1293840000000L)));
+ assertEquals(expectedResultList, resultList);
+
+ // Test 2 : two splits, create 4
+ intervals = new ArrayList<>();
+ intervals.add(new Interval(1262304000000L, 1293840000000L));
+ intervals.add(new Interval(1325376000000L, 1356998400000L));
+ resultList = (List<List<Interval>>) method1.invoke(input, intervals, 4);
+ expectedResultList = new ArrayList<>();
+ expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1278093600000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1278093600000L, 1293840000000L),
+ new Interval(1325376000000L, 1325419200000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1325419200000L, 1341208800000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1341208800000L, 1356998400000L)));
+ assertEquals(expectedResultList, resultList);
+
+ // Test 3 : two splits, create 5
+ intervals = new ArrayList<>();
+ intervals.add(new Interval(1262304000000L, 1293840000000L));
+ intervals.add(new Interval(1325376000000L, 1356998400000L));
+ resultList = (List<List<Interval>>) method1.invoke(input, intervals, 5);
+ expectedResultList = new ArrayList<>();
+ expectedResultList.add(Arrays.asList(new Interval(1262304000000L, 1274935680000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1274935680000L, 1287567360000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1287567360000L, 1293840000000L),
+ new Interval(1325376000000L, 1331735040000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1331735040000L, 1344366720000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1344366720000L, 1356998400000L)));
+ assertEquals(expectedResultList, resultList);
+
+ // Test 4 : three splits, different ranges, create 6
+ intervals = new ArrayList<>();
+ intervals.add(new Interval(1199145600000L, 1201824000000L)); // one month
+ intervals.add(new Interval(1325376000000L, 1356998400000L)); // one year
+ intervals.add(new Interval(1407283200000L, 1407888000000L)); // 7 days
+ resultList = (List<List<Interval>>) method1.invoke(input, intervals, 6);
+ expectedResultList = new ArrayList<>();
+ expectedResultList.add(Arrays.asList(new Interval(1199145600000L, 1201824000000L),
+ new Interval(1325376000000L, 1328515200000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1328515200000L, 1334332800000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1334332800000L, 1340150400000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1340150400000L, 1345968000000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1345968000000L, 1351785600000L)));
+ expectedResultList.add(Arrays.asList(new Interval(1351785600000L, 1356998400000L),
+ new Interval(1407283200000L, 1407888000000L)));
+ assertEquals(expectedResultList, resultList);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/itests/qtest/pom.xml
----------------------------------------------------------------------
diff --git a/itests/qtest/pom.xml b/itests/qtest/pom.xml
index 7fc72b9..e762d0e 100644
--- a/itests/qtest/pom.xml
+++ b/itests/qtest/pom.xml
@@ -110,6 +110,19 @@
<version>${hadoop.version}</version>
<optional>true</optional>
</dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-druid-handler</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-druid-handler</artifactId>
+ <version>${project.version}</version>
+ <classifier>tests</classifier>
+ <scope>test</scope>
+ </dependency>
<!-- test inter-project -->
<dependency>
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/packaging/pom.xml
----------------------------------------------------------------------
diff --git a/packaging/pom.xml b/packaging/pom.xml
index 679dfe8..76e0cff 100644
--- a/packaging/pom.xml
+++ b/packaging/pom.xml
@@ -210,6 +210,11 @@
</dependency>
<dependency>
<groupId>org.apache.hive</groupId>
+ <artifactId>hive-druid-handler</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hive</groupId>
<artifactId>hive-hwi</artifactId>
<version>${project.version}</version>
</dependency>
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4c41200..2fb78cd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -37,6 +37,7 @@
<module>cli</module>
<module>common</module>
<module>contrib</module>
+ <module>druid-handler</module>
<module>hbase-handler</module>
<module>hcatalog</module>
<module>hplsql</module>
@@ -130,6 +131,7 @@
<derby.version>10.10.2.0</derby.version>
<dropwizard.version>3.1.0</dropwizard.version>
<dropwizard-metrics-hadoop-metrics2-reporter.version>0.1.2</dropwizard-metrics-hadoop-metrics2-reporter.version>
+ <druid.version>0.9.1.1</druid.version>
<guava.version>14.0.1</guava.version>
<groovy.version>2.4.4</groovy.version>
<hadoop.version>2.7.2</hadoop.version>
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 5722305..66cbdd3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.conf.Constants;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.io.HdfsUtils;
@@ -4468,12 +4469,13 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
}
public static boolean doesTableNeedLocation(Table tbl) {
- // If we are ok with breaking compatibility of existing 3rd party StorageHandlers,
+ // TODO: If we are ok with breaking compatibility of existing 3rd party StorageHandlers,
// this method could be moved to the HiveStorageHandler interface.
boolean retval = true;
if (tbl.getStorageHandler() != null) {
- retval = !tbl.getStorageHandler().toString().equals(
- "org.apache.hadoop.hive.hbase.HBaseStorageHandler");
+ String sh = tbl.getStorageHandler().toString();
+ retval = !sh.equals("org.apache.hadoop.hive.hbase.HBaseStorageHandler")
+ && !sh.equals(Constants.DRUID_HIVE_STORAGE_HANDLER_ID);
}
return retval;
}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 60646ba..4710b8f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -31,14 +31,20 @@ import java.util.TreeSet;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.hive.ql.exec.FunctionInfo.FunctionResource;
import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.parse.SemanticException;
import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
import org.apache.hadoop.hive.ql.plan.ExprNodeGenericFuncDesc;
import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorDay;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorHour;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMinute;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorMonth;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorQuarter;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorSecond;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorWeek;
+import org.apache.hadoop.hive.ql.udf.UDFDateFloorYear;
import org.apache.hadoop.hive.ql.udf.SettableUDF;
import org.apache.hadoop.hive.ql.udf.UDAFPercentile;
import org.apache.hadoop.hive.ql.udf.UDFAcos;
@@ -141,6 +147,8 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hive.common.util.AnnotationUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* FunctionRegistry.
@@ -289,6 +297,16 @@ public final class FunctionRegistry {
system.registerGenericUDF("trunc", GenericUDFTrunc.class);
system.registerGenericUDF("date_format", GenericUDFDateFormat.class);
+ // Special date formatting functions
+ system.registerUDF("floor_year", UDFDateFloorYear.class, false);
+ system.registerUDF("floor_quarter", UDFDateFloorQuarter.class, false);
+ system.registerUDF("floor_month", UDFDateFloorMonth.class, false);
+ system.registerUDF("floor_day", UDFDateFloorDay.class, false);
+ system.registerUDF("floor_week", UDFDateFloorWeek.class, false);
+ system.registerUDF("floor_hour", UDFDateFloorHour.class, false);
+ system.registerUDF("floor_minute", UDFDateFloorMinute.class, false);
+ system.registerUDF("floor_second", UDFDateFloorSecond.class, false);
+
system.registerGenericUDF("date_add", GenericUDFDateAdd.class);
system.registerGenericUDF("date_sub", GenericUDFDateSub.class);
system.registerGenericUDF("datediff", GenericUDFDateDiff.class);
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
index aeb4e7d..890aea1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/HivePlannerContext.java
@@ -19,21 +19,28 @@ package org.apache.hadoop.hive.ql.optimizer.calcite;
import org.apache.calcite.plan.Context;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveRulesRegistry;
public class HivePlannerContext implements Context {
- private HiveAlgorithmsConf config;
+ private HiveAlgorithmsConf algoConfig;
+ private HiveDruidConf druidConf;
private HiveRulesRegistry registry;
- public HivePlannerContext(HiveAlgorithmsConf config, HiveRulesRegistry registry) {
- this.config = config;
+ public HivePlannerContext(HiveAlgorithmsConf algoConfig, HiveDruidConf druidConf,
+ HiveRulesRegistry registry) {
+ this.algoConfig = algoConfig;
+ this.druidConf = druidConf;
this.registry = registry;
}
public <T> T unwrap(Class<T> clazz) {
- if (clazz.isInstance(config)) {
- return clazz.cast(config);
+ if (clazz.isInstance(algoConfig)) {
+ return clazz.cast(algoConfig);
+ }
+ if (clazz.isInstance(druidConf)) {
+ return clazz.cast(druidConf);
}
if (clazz.isInstance(registry)) {
return clazz.cast(registry);
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
new file mode 100644
index 0000000..82ab4d7
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidIntervalUtils.java
@@ -0,0 +1,466 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+import java.sql.Timestamp;
+import java.text.DateFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.TreeSet;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.commons.lang.StringUtils;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.collect.BoundType;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Range;
+import com.google.common.collect.Sets;
+
+/**
+ * Utilities for generating intervals from RexNode.
+ *
+ * Based on Navis logic implemented on Hive data structures.
+ * See <a href="https://github.com/druid-io/druid/pull/2880">Druid PR-2880</a>
+ *
+ */
+@SuppressWarnings({"rawtypes","unchecked"})
+public class DruidIntervalUtils {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(DruidIntervalUtils.class);
+
+
+ /**
+ * Given a list of predicates, it generates the equivalent Interval
+ * (if possible). It assumes that all the predicates in the input
+ * reference a single column : the timestamp column.
+ *
+ * @param conjs list of conditions to use for the transformation
+ * @return interval representing the conditions in the input list
+ */
+ public static List<Interval> createInterval(RelDataType type, List<RexNode> conjs) {
+ List<Range> ranges = new ArrayList<>();
+ for (RexNode child : conjs) {
+ List<Range> extractedRanges = extractRanges(type, child, false);
+ if (extractedRanges == null || extractedRanges.isEmpty()) {
+ // We could not extract, we bail out
+ return null;
+ }
+ if (ranges.isEmpty()) {
+ ranges.addAll(extractedRanges);
+ continue;
+ }
+ List<Range> overlapped = Lists.newArrayList();
+ for (Range current : ranges) {
+ for (Range interval : extractedRanges) {
+ if (current.isConnected(interval)) {
+ overlapped.add(current.intersection(interval));
+ }
+ }
+ }
+ ranges = overlapped;
+ }
+ List<Range> compactRanges = condenseRanges(ranges);
+ LOG.debug("Inferred ranges on interval : " + compactRanges);
+ return toInterval(compactRanges);
+ }
+
+ protected static List<Interval> toInterval(List<Range> ranges) {
+ List<Interval> intervals = Lists.transform(ranges, new Function<Range, Interval>() {
+ @Override
+ public Interval apply(Range range) {
+ if (!range.hasLowerBound() && !range.hasUpperBound()) {
+ return DruidTable.DEFAULT_INTERVAL;
+ }
+ long start = range.hasLowerBound() ? toLong(range.lowerEndpoint()) :
+ DruidTable.DEFAULT_INTERVAL.getStartMillis();
+ long end = range.hasUpperBound() ? toLong(range.upperEndpoint()) :
+ DruidTable.DEFAULT_INTERVAL.getEndMillis();
+ if (range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN) {
+ start++;
+ }
+ if (range.hasUpperBound() && range.upperBoundType() == BoundType.CLOSED) {
+ end++;
+ }
+ return new Interval(start, end);
+ }
+ });
+ LOG.info("Converted time ranges " + ranges + " to interval " + intervals);
+ return intervals;
+ }
+
+ protected static List<Range> extractRanges(RelDataType type, RexNode node,
+ boolean withNot) {
+ switch (node.getKind()) {
+ case EQUALS:
+ case LESS_THAN:
+ case LESS_THAN_OR_EQUAL:
+ case GREATER_THAN:
+ case GREATER_THAN_OR_EQUAL:
+ case BETWEEN:
+ case IN:
+ return leafToRanges(type, (RexCall) node, withNot);
+
+ case NOT:
+ return extractRanges(type, ((RexCall) node).getOperands().get(0), !withNot);
+
+ case OR:
+ RexCall call = (RexCall) node;
+ List<Range> intervals = Lists.newArrayList();
+ for (RexNode child : call.getOperands()) {
+ List<Range> extracted = extractRanges(type, child, withNot);
+ if (extracted != null) {
+ intervals.addAll(extracted);
+ }
+ }
+ return intervals;
+
+ default:
+ return null;
+ }
+ }
+
+ protected static List<Range> leafToRanges(RelDataType type, RexCall call,
+ boolean withNot) {
+ switch (call.getKind()) {
+ case EQUALS:
+ case LESS_THAN:
+ case LESS_THAN_OR_EQUAL:
+ case GREATER_THAN:
+ case GREATER_THAN_OR_EQUAL:
+ {
+ RexLiteral literal = null;
+ if (call.getOperands().get(0) instanceof RexInputRef &&
+ call.getOperands().get(1) instanceof RexLiteral) {
+ literal = extractLiteral(call.getOperands().get(1));
+ } else if (call.getOperands().get(0) instanceof RexInputRef &&
+ call.getOperands().get(1).getKind() == SqlKind.CAST) {
+ literal = extractLiteral(call.getOperands().get(1));
+ } else if (call.getOperands().get(1) instanceof RexInputRef &&
+ call.getOperands().get(0) instanceof RexLiteral) {
+ literal = extractLiteral(call.getOperands().get(0));
+ } else if (call.getOperands().get(1) instanceof RexInputRef &&
+ call.getOperands().get(0).getKind() == SqlKind.CAST) {
+ literal = extractLiteral(call.getOperands().get(0));
+ }
+ if (literal == null) {
+ return null;
+ }
+ Comparable value = literalToType(literal, type);
+ if (value == null) {
+ return null;
+ }
+ if (call.getKind() == SqlKind.LESS_THAN) {
+ return Arrays.<Range> asList(withNot ? Range.atLeast(value) : Range.lessThan(value));
+ } else if (call.getKind() == SqlKind.LESS_THAN_OR_EQUAL) {
+ return Arrays.<Range> asList(withNot ? Range.greaterThan(value) : Range.atMost(value));
+ } else if (call.getKind() == SqlKind.GREATER_THAN) {
+ return Arrays.<Range> asList(withNot ? Range.atMost(value) : Range.greaterThan(value));
+ } else if (call.getKind() == SqlKind.GREATER_THAN_OR_EQUAL) {
+ return Arrays.<Range> asList(withNot ? Range.lessThan(value) : Range.atLeast(value));
+ } else { //EQUALS
+ if (!withNot) {
+ return Arrays.<Range> asList(Range.closed(value, value));
+ }
+ return Arrays.<Range> asList(Range.lessThan(value), Range.greaterThan(value));
+ }
+ }
+ case BETWEEN:
+ {
+ RexLiteral literal1 = extractLiteral(call.getOperands().get(2));
+ if (literal1 == null) {
+ return null;
+ }
+ RexLiteral literal2 = extractLiteral(call.getOperands().get(3));
+ if (literal2 == null) {
+ return null;
+ }
+ Comparable value1 = literalToType(literal1, type);
+ Comparable value2 = literalToType(literal2, type);
+ if (value1 == null || value2 == null) {
+ return null;
+ }
+ boolean inverted = value1.compareTo(value2) > 0;
+ if (!withNot) {
+ return Arrays.<Range> asList(
+ inverted ? Range.closed(value2, value1) : Range.closed(value1, value2));
+ }
+ return Arrays.<Range> asList(Range.lessThan(inverted ? value2 : value1),
+ Range.greaterThan(inverted ? value1 : value2));
+ }
+ case IN:
+ {
+ List<Range> ranges = Lists.newArrayList();
+ for (int i = 1; i < call.getOperands().size(); i++) {
+ RexLiteral literal = extractLiteral(call.getOperands().get(i));
+ if (literal == null) {
+ return null;
+ }
+ Comparable element = literalToType(literal, type);
+ if (element == null) {
+ return null;
+ }
+ if (withNot) {
+ ranges.addAll(
+ Arrays.<Range> asList(Range.lessThan(element), Range.greaterThan(element)));
+ } else {
+ ranges.add(Range.closed(element, element));
+ }
+ }
+ return ranges;
+ }
+ default:
+ return null;
+ }
+ }
+
+ @SuppressWarnings("incomplete-switch")
+ protected static Comparable literalToType(RexLiteral literal, RelDataType type) {
+ // Extract
+ Object value = null;
+ switch (literal.getType().getSqlTypeName()) {
+ case DATE:
+ case TIME:
+ case TIMESTAMP:
+ case INTERVAL_YEAR_MONTH:
+ case INTERVAL_DAY_TIME:
+ value = literal.getValue();
+ break;
+ case TINYINT:
+ case SMALLINT:
+ case INTEGER:
+ case BIGINT:
+ case DOUBLE:
+ case DECIMAL:
+ case FLOAT:
+ case REAL:
+ case VARCHAR:
+ case CHAR:
+ case BOOLEAN:
+ value = literal.getValue3();
+ }
+ if (value == null) {
+ return null;
+ }
+
+ // Convert
+ switch (type.getSqlTypeName()) {
+ case BIGINT:
+ return toLong(value);
+ case INTEGER:
+ return toInt(value);
+ case FLOAT:
+ return toFloat(value);
+ case DOUBLE:
+ return toDouble(value);
+ case VARCHAR:
+ case CHAR:
+ return String.valueOf(value);
+ case TIMESTAMP:
+ return toTimestamp(value);
+ }
+ return null;
+ }
+
+ private static RexLiteral extractLiteral(RexNode node) {
+ RexNode target = node;
+ if (node.getKind() == SqlKind.CAST) {
+ target = ((RexCall)node).getOperands().get(0);
+ }
+ if (!(target instanceof RexLiteral)) {
+ return null;
+ }
+ return (RexLiteral) target;
+ }
+
+ private static Comparable toTimestamp(Object literal) {
+ if (literal instanceof Timestamp) {
+ return (Timestamp) literal;
+ }
+ if (literal instanceof Date) {
+ return new Timestamp(((Date) literal).getTime());
+ }
+ if (literal instanceof Number) {
+ return new Timestamp(((Number) literal).longValue());
+ }
+ if (literal instanceof String) {
+ String string = (String) literal;
+ if (StringUtils.isNumeric(string)) {
+ return new Timestamp(Long.valueOf(string));
+ }
+ try {
+ return Timestamp.valueOf(string);
+ } catch (NumberFormatException e) {
+ // ignore
+ }
+ }
+ return null;
+ }
+
+ private static Long toLong(Object literal) {
+ if (literal instanceof Number) {
+ return ((Number) literal).longValue();
+ }
+ if (literal instanceof Date) {
+ return ((Date) literal).getTime();
+ }
+ if (literal instanceof Timestamp) {
+ return ((Timestamp) literal).getTime();
+ }
+ if (literal instanceof String) {
+ try {
+ return Long.valueOf((String) literal);
+ } catch (NumberFormatException e) {
+ // ignore
+ }
+ try {
+ return DateFormat.getDateInstance().parse((String) literal).getTime();
+ } catch (ParseException e) {
+ // best effort. ignore
+ }
+ }
+ return null;
+ }
+
+
+ private static Integer toInt(Object literal) {
+ if (literal instanceof Number) {
+ return ((Number) literal).intValue();
+ }
+ if (literal instanceof String) {
+ try {
+ return Integer.valueOf((String) literal);
+ } catch (NumberFormatException e) {
+ // ignore
+ }
+ }
+ return null;
+ }
+
+ private static Float toFloat(Object literal) {
+ if (literal instanceof Number) {
+ return ((Number) literal).floatValue();
+ }
+ if (literal instanceof String) {
+ try {
+ return Float.valueOf((String) literal);
+ } catch (NumberFormatException e) {
+ // ignore
+ }
+ }
+ return null;
+ }
+
+ private static Double toDouble(Object literal) {
+ if (literal instanceof Number) {
+ return ((Number) literal).doubleValue();
+ }
+ if (literal instanceof String) {
+ try {
+ return Double.valueOf((String) literal);
+ } catch (NumberFormatException e) {
+ // ignore
+ }
+ }
+ return null;
+ }
+
+ protected static List<Range> condenseRanges(List<Range> ranges) {
+ if (ranges.size() <= 1) {
+ return ranges;
+ }
+
+ Comparator<Range> startThenEnd = new Comparator<Range>() {
+ @Override
+ public int compare(Range lhs, Range rhs) {
+ int compare = 0;
+ if (lhs.hasLowerBound() && rhs.hasLowerBound()) {
+ compare = lhs.lowerEndpoint().compareTo(rhs.lowerEndpoint());
+ } else if (!lhs.hasLowerBound() && rhs.hasLowerBound()) {
+ compare = -1;
+ } else if (lhs.hasLowerBound() && !rhs.hasLowerBound()) {
+ compare = 1;
+ }
+ if (compare != 0) {
+ return compare;
+ }
+ if (lhs.hasUpperBound() && rhs.hasUpperBound()) {
+ compare = lhs.upperEndpoint().compareTo(rhs.upperEndpoint());
+ } else if (!lhs.hasUpperBound() && rhs.hasUpperBound()) {
+ compare = -1;
+ } else if (lhs.hasUpperBound() && !rhs.hasUpperBound()) {
+ compare = 1;
+ }
+ return compare;
+ }
+ };
+
+ TreeSet<Range> sortedIntervals = Sets.newTreeSet(startThenEnd);
+ sortedIntervals.addAll(ranges);
+
+ List<Range> retVal = Lists.newArrayList();
+
+ Iterator<Range> intervalsIter = sortedIntervals.iterator();
+ Range currInterval = intervalsIter.next();
+ while (intervalsIter.hasNext()) {
+ Range next = intervalsIter.next();
+ if (currInterval.encloses(next)) {
+ continue;
+ }
+ if (mergeable(currInterval, next)) {
+ currInterval = currInterval.span(next);
+ } else {
+ retVal.add(currInterval);
+ currInterval = next;
+ }
+ }
+ retVal.add(currInterval);
+
+ return retVal;
+ }
+
+ protected static boolean mergeable(Range range1, Range range2) {
+ Comparable x1 = range1.upperEndpoint();
+ Comparable x2 = range2.lowerEndpoint();
+ int compare = x1.compareTo(x2);
+ return compare > 0 || (compare == 0 && range1.upperBoundType() == BoundType.CLOSED
+ && range2.lowerBoundType() == BoundType.CLOSED);
+ }
+
+ public static long extractTotalTime(List<Interval> intervals) {
+ long totalTime = 0;
+ for (Interval interval : intervals) {
+ totalTime += (interval.getEndMillis() - interval.getStartMillis());
+ }
+ return totalTime;
+ }
+
+}
[2/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_basic2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_basic2.q.out b/ql/src/test/results/clientpositive/druid_basic2.q.out
new file mode 100644
index 0000000..3205905
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_basic2.q.out
@@ -0,0 +1,533 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: -- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Select Operator
+ expressions: robot (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":["delta"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Select Operator
+ expressions: delta (type: float)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Select Operator
+ expressions: robot (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en'
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"ALL","dimensions":["robot"],"limitSpec":{"type":"default"},"filter":{"type":"selector","dimension":"language","value":"en"},"aggregations":[{"type":"longSum","name":"dummy_agg","fieldName":"dummy_agg"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type groupBy
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Select Operator
+ expressions: robot (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1) a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1) a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ filterExpr: language is not null (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: language is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), language (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col1 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col1 (type: string)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ tag: 0
+ value expressions: _col0 (type: string)
+ auto parallelism: false
+ TableScan
+ alias: druid_table_1
+ filterExpr: language is not null (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Filter Operator
+ isSamplingPred: false
+ predicate: language is not null (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: language (type: string)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string)
+ null sort order: a
+ sort order: +
+ Map-reduce partition columns: _col0 (type: string)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ tag: 1
+ auto parallelism: false
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: druid_table_1
+ input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+ output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+ EXTERNAL TRUE
+ bucket_count -1
+ columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+ columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+ columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+ druid.datasource wikipedia
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+#### A masked pattern was here ####
+ name default.druid_table_1
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+
+ input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+ output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+ EXTERNAL TRUE
+ bucket_count -1
+ columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+ columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+ columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+ druid.datasource wikipedia
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+#### A masked pattern was here ####
+ name default.druid_table_1
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+ name: default.druid_table_1
+ name: default.druid_table_1
+ Truncated Path -> Alias:
+ /druid_table_1 [$hdt$_0:druid_table_1, $hdt$_1:druid_table_1]
+ Needs Tagging: true
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0 _col1 (type: string)
+ 1 _col0 (type: string)
+ outputColumnNames: _col0, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: _col0 (type: string), _col2 (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+#### A masked pattern was here ####
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ properties:
+ columns _col0,_col1
+ columns.types string:string
+ escape.delim \
+ hive.serialization.extend.additional.nesting.levels true
+ serialization.escape.crlf true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+Warning: Shuffle Join JOIN[5][tables = [druid_table_1, $hdt$_0]] in Stage 'Stage-1:MAPRED' is a cross product
+PREHOOK: query: EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1
+ WHERE language = 'en') a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+)
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1
+ WHERE language = 'en') a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ GatherStats: false
+ Select Operator
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
+ tag: 1
+ auto parallelism: false
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ GatherStats: false
+ Reduce Output Operator
+ null sort order:
+ sort order:
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ tag: 0
+ value expressions: robot (type: string)
+ auto parallelism: false
+ Path -> Alias:
+#### A masked pattern was here ####
+ Path -> Partition:
+#### A masked pattern was here ####
+ Partition
+ base file name: druid_table_1
+ input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+ output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+ EXTERNAL TRUE
+ bucket_count -1
+ columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+ columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+ columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+ druid.datasource wikipedia
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+#### A masked pattern was here ####
+ name default.druid_table_1
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+
+ input format: org.apache.hadoop.hive.druid.HiveDruidQueryBasedInputFormat
+ output format: org.apache.hadoop.hive.druid.HiveDruidOutputFormat
+ properties:
+ COLUMN_STATS_ACCURATE {"BASIC_STATS":"true"}
+ EXTERNAL TRUE
+ bucket_count -1
+ columns __time,robot,namespace,anonymous,unpatrolled,page,language,newpage,user,count,added,delta,variation,deleted
+ columns.comments 'from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer','from deserializer'
+ columns.types timestamp:string:string:string:string:string:string:string:string:float:float:float:float:float
+ druid.datasource wikipedia
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"filter":{"type":"selector","dimension":"language","value":"en"},"dimensions":["robot"],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+#### A masked pattern was here ####
+ name default.druid_table_1
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ serialization.ddl struct druid_table_1 { timestamp __time, string robot, string namespace, string anonymous, string unpatrolled, string page, string language, string newpage, string user, float count, float added, float delta, float variation, float deleted}
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.druid.QTestDruidSerDe
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+ serde: org.apache.hadoop.hive.druid.QTestDruidSerDe
+ name: default.druid_table_1
+ name: default.druid_table_1
+ Truncated Path -> Alias:
+ /druid_table_1 [$hdt$_0:druid_table_1, druid_table_1]
+ Needs Tagging: true
+ Reduce Operator Tree:
+ Join Operator
+ condition map:
+ Inner Join 0 to 1
+ keys:
+ 0
+ 1
+ outputColumnNames: _col1
+ Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+ Select Operator
+ expressions: _col1 (type: string), 'en' (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+ File Output Operator
+ compressed: false
+ GlobalTableId: 0
+#### A masked pattern was here ####
+ NumFilesPerFileSink: 1
+ Statistics: Num rows: 1 Data size: 1 Basic stats: COMPLETE Column stats: NONE
+#### A masked pattern was here ####
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ properties:
+ columns _col0,_col1
+ columns.types string:string
+ escape.delim \
+ hive.serialization.extend.additional.nesting.levels true
+ serialization.escape.crlf true
+ serialization.format 1
+ serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+ TotalFiles: 1
+ GatherStats: false
+ MultiFileSpray: false
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_intervals.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_intervals.q.out b/ql/src/test/results/clientpositive/druid_intervals.q.out
new file mode 100644
index 0000000..984bb79
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_intervals.q.out
@@ -0,0 +1,398 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: -- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-03-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+ AND `__time` < '2011-01-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+ AND `__time` < '2011-01-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.000-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00'
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00'
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00","2012-01-01T00:00:00.000-08:00/2013-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2012-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: -- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp)
+ outputColumnNames: _col0
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["2010-01-01T00:00:00.000-08:00/2010-01-01T00:00:00.001-08:00","2011-01-01T00:00:00.000-08:00/2011-01-01T00:00:00.001-08:00"],"filter":{"type":"selector","dimension":"robot","value":"user1"},"dimensions":[],"metrics":[],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), 'user1' (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+PREHOOK: type: QUERY
+POSTHOOK: query: EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00')
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ filterExpr: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: ((__time) IN ('2010-01-01 00:00:00', '2011-01-01 00:00:00') or (robot = 'user1')) (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), robot (type: string)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_timeseries.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_timeseries.q.out b/ql/src/test/results/clientpositive/druid_timeseries.q.out
new file mode 100644
index 0000000..8d974a4
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_timeseries.q.out
@@ -0,0 +1,566 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"ALL","aggregations":[{"type":"longMax","name":"$f0","fieldName":"added"},{"type":"doubleSum","name":"$f1","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: $f0 (type: bigint), $f1 (type: float)
+ outputColumnNames: _col0, _col1
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"NONE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"YEAR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"QUARTER","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MONTH","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"WEEK","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"DAY","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"MINUTE","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"SECOND","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"timeseries","dataSource":"wikipedia","descending":"false","granularity":"HOUR","filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type timeseries
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: __time (type: timestamp), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: max(_col1), sum(_col2)
+ keys: _col0 (type: timestamp)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: timestamp)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: timestamp)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ value expressions: _col1 (type: float), _col2 (type: double)
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: max(VALUE._col0), sum(VALUE._col1)
+ keys: KEY._col0 (type: timestamp)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
+PREHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+ SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+ FROM druid_table_1
+ GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+ SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+ FROM druid_table_1
+ GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-0 depends on stages: Stage-1
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: max(_col1), sum(_col2)
+ keys: _col0 (type: timestamp)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: timestamp)
+ sort order: +
+ Map-reduce partition columns: _col0 (type: timestamp)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ value expressions: _col1 (type: float), _col2 (type: double)
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: max(VALUE._col0), sum(VALUE._col1)
+ keys: KEY._col0 (type: timestamp)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ ListSink
+
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_topn.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_topn.q.out b/ql/src/test/results/clientpositive/druid_topn.q.out
new file mode 100644
index 0000000..17bdaed
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_topn.q.out
@@ -0,0 +1,419 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"ALL","dimension":"robot","metric":"$f1","aggregations":[{"type":"longMax","name":"$f1","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
+ druid.query.type topN
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), $f1 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"NONE","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":100}
+ druid.query.type topN
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"topN","dataSource":"wikipedia","granularity":"YEAR","dimension":"robot","metric":"$f3","aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"threshold":10}
+ druid.query.type topN
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f3","direction":"ascending"}]},"aggregations":[{"type":"longMax","name":"$f2","fieldName":"added"},{"type":"doubleSum","name":"$f3","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type groupBy
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), __time (type: timestamp), $f2 (type: bigint), $f3 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f4","direction":"descending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type groupBy
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"MONTH","dimensions":["robot","namespace"],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"robot","direction":"ascending"},{"dimension":"$f3","direction":"descending"}]},"aggregations":[{"type":"longMax","name":"$f3","fieldName":"added"},{"type":"doubleSum","name":"$f4","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type groupBy
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), __time (type: timestamp), $f3 (type: bigint), $f4 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Fetch Operator
+ limit: -1
+ Processor Tree:
+ TableScan
+ alias: druid_table_1
+ properties:
+ druid.query.json {"queryType":"groupBy","dataSource":"wikipedia","granularity":"YEAR","dimensions":[],"limitSpec":{"type":"default","limit":10,"columns":[{"dimension":"$f2","direction":"ascending"}]},"filter":{"type":"selector","dimension":"robot","value":"1"},"aggregations":[{"type":"longMax","name":"$f1_0","fieldName":"added"},{"type":"doubleSum","name":"$f2","fieldName":"variation"}],"intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"]}
+ druid.query.type groupBy
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: '1' (type: string), __time (type: timestamp), $f1_0 (type: bigint), $f2 (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ ListSink
+
+PREHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100
+PREHOOK: type: QUERY
+POSTHOOK: query: -- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100
+POSTHOOK: type: QUERY
+STAGE DEPENDENCIES:
+ Stage-1 is a root stage
+ Stage-2 depends on stages: Stage-1
+ Stage-0 depends on stages: Stage-2
+
+STAGE PLANS:
+ Stage: Stage-1
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ alias: druid_table_1
+ filterExpr: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ properties:
+ druid.query.json {"queryType":"select","dataSource":"wikipedia","descending":"false","intervals":["1900-01-01T00:00:00.000-08:00/3000-01-01T00:00:00.000-08:00"],"dimensions":["robot","namespace","anonymous","unpatrolled","page","language","newpage","user"],"metrics":["count","added","delta","variation","deleted"],"granularity":"ALL","pagingSpec":{"threshold":1},"context":{"druid.query.fetch":false}}
+ druid.query.type select
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Filter Operator
+ predicate: floor_hour(__time) BETWEEN 2010-01-01 00:00:00.0 AND 2014-01-01 00:00:00.0 (type: boolean)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Select Operator
+ expressions: robot (type: string), floor_hour(__time) (type: timestamp), added (type: float), variation (type: float)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Group By Operator
+ aggregations: max(_col2), sum(_col3)
+ keys: _col0 (type: string), _col1 (type: timestamp)
+ mode: hash
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Reduce Output Operator
+ key expressions: _col0 (type: string), _col1 (type: timestamp)
+ sort order: ++
+ Map-reduce partition columns: _col0 (type: string), _col1 (type: timestamp)
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ value expressions: _col2 (type: float), _col3 (type: double)
+ Reduce Operator Tree:
+ Group By Operator
+ aggregations: max(VALUE._col0), sum(VALUE._col1)
+ keys: KEY._col0 (type: string), KEY._col1 (type: timestamp)
+ mode: mergepartial
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+
+ Stage: Stage-2
+ Map Reduce
+ Map Operator Tree:
+ TableScan
+ Reduce Output Operator
+ key expressions: _col2 (type: float)
+ sort order: +
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ TopN Hash Memory Usage: 0.1
+ value expressions: _col0 (type: string), _col1 (type: timestamp), _col3 (type: double)
+ Reduce Operator Tree:
+ Select Operator
+ expressions: VALUE._col0 (type: string), VALUE._col1 (type: timestamp), KEY.reducesinkkey0 (type: float), VALUE._col2 (type: double)
+ outputColumnNames: _col0, _col1, _col2, _col3
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ Limit
+ Number of rows: 100
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ File Output Operator
+ compressed: false
+ Statistics: Num rows: 1 Data size: 0 Basic stats: PARTIAL Column stats: NONE
+ table:
+ input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+ output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+ serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+
+ Stage: Stage-0
+ Fetch Operator
+ limit: 100
+ Processor Tree:
+ ListSink
+
[4/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
new file mode 100644
index 0000000..43982aa
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQuery.java
@@ -0,0 +1,1053 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+import java.io.IOException;
+import java.io.StringWriter;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveProject;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.JsonFactory;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+
+/**
+ * Relational expression representing a scan of a Druid data set.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidQuery extends TableScan {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(DruidQuery.class);
+
+ protected QuerySpec querySpec;
+
+ final DruidTable druidTable;
+ final List<Interval> intervals;
+ final ImmutableList<RelNode> rels;
+
+ private static final Pattern VALID_SIG = Pattern.compile("sf?p?a?l?");
+
+ /**
+ * Creates a DruidQuery.
+ *
+ * @param cluster Cluster
+ * @param traitSet Traits
+ * @param table Table
+ * @param druidTable Druid table
+ * @param interval Interval for the query
+ * @param rels Internal relational expressions
+ */
+ private DruidQuery(RelOptCluster cluster, RelTraitSet traitSet,
+ RelOptTable table, DruidTable druidTable,
+ List<Interval> intervals, List<RelNode> rels) {
+ super(cluster, traitSet, table);
+ this.druidTable = druidTable;
+ this.intervals = ImmutableList.copyOf(intervals);
+ this.rels = ImmutableList.copyOf(rels);
+
+ assert isValid(Litmus.THROW);
+ }
+
+ /** Returns a string describing the operations inside this query.
+ *
+ * <p>For example, "sfpal" means {@link TableScan} (s)
+ * followed by {@link Filter} (f)
+ * followed by {@link Project} (p)
+ * followed by {@link Aggregate} (a)
+ * followed by {@link Sort} (l).
+ *
+ * @see #isValidSignature(String)
+ */
+ String signature() {
+ final StringBuilder b = new StringBuilder();
+ for (RelNode rel : rels) {
+ b.append(rel instanceof TableScan ? 's'
+ : rel instanceof Project ? 'p'
+ : rel instanceof Filter ? 'f'
+ : rel instanceof Aggregate ? 'a'
+ : rel instanceof Sort ? 'l'
+ : '!');
+ }
+ return b.toString();
+ }
+
+ @Override public boolean isValid(Litmus litmus) {
+ if (!super.isValid(litmus)) {
+ return false;
+ }
+ final String signature = signature();
+ if (!isValidSignature(signature)) {
+ return litmus.fail("invalid signature");
+ }
+ if (rels.isEmpty()) {
+ return litmus.fail("must have at least one rel");
+ }
+ for (int i = 0; i < rels.size(); i++) {
+ final RelNode r = rels.get(i);
+ if (i == 0) {
+ if (!(r instanceof TableScan)) {
+ return litmus.fail("first rel must be TableScan");
+ }
+ if (r.getTable() != table) {
+ return litmus.fail("first rel must be based on table table");
+ }
+ } else {
+ final List<RelNode> inputs = r.getInputs();
+ if (inputs.size() != 1 || inputs.get(0) != rels.get(i - 1)) {
+ return litmus.fail("each rel must have a single input");
+ }
+ if (r instanceof Aggregate) {
+ final Aggregate aggregate = (Aggregate) r;
+ if (aggregate.getGroupSets().size() != 1
+ || aggregate.indicator) {
+ return litmus.fail("no grouping sets");
+ }
+ for (AggregateCall call : aggregate.getAggCallList()) {
+ if (call.filterArg >= 0) {
+ return litmus.fail("no filtered aggregate functions");
+ }
+ }
+ }
+ if (r instanceof Filter) {
+ final Filter filter = (Filter) r;
+ if (!isValidFilter(filter.getCondition())) {
+ return litmus.fail("invalid filter");
+ }
+ }
+ if (r instanceof Sort) {
+ final Sort sort = (Sort) r;
+ if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
+ return litmus.fail("offset not supported");
+ }
+ }
+ }
+ }
+ return true;
+ }
+
+ boolean isValidFilter(RexNode e) {
+ switch (e.getKind()) {
+ case INPUT_REF:
+ case LITERAL:
+ return true;
+ case AND:
+ case OR:
+ case NOT:
+ case EQUALS:
+ case LESS_THAN:
+ case LESS_THAN_OR_EQUAL:
+ case GREATER_THAN:
+ case GREATER_THAN_OR_EQUAL:
+ case BETWEEN:
+ case IN:
+ case CAST:
+ return areValidFilters(((RexCall) e).getOperands());
+ default:
+ return false;
+ }
+ }
+
+ private boolean areValidFilters(List<RexNode> es) {
+ for (RexNode e : es) {
+ if (!isValidFilter(e)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /** Returns whether a signature represents an sequence of relational operators
+ * that can be translated into a valid Druid query. */
+ static boolean isValidSignature(String signature) {
+ return VALID_SIG.matcher(signature).matches();
+ }
+
+ /** Creates a DruidQuery. */
+ public static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
+ RelOptTable table, DruidTable druidTable, List<RelNode> rels) {
+ return new DruidQuery(cluster, traitSet, table, druidTable, druidTable.intervals, rels);
+ }
+
+ /** Creates a DruidQuery. */
+ private static DruidQuery create(RelOptCluster cluster, RelTraitSet traitSet,
+ RelOptTable table, DruidTable druidTable, List<Interval> intervals, List<RelNode> rels) {
+ return new DruidQuery(cluster, traitSet, table, druidTable, intervals, rels);
+ }
+
+ /** Extends a DruidQuery. */
+ public static DruidQuery extendQuery(DruidQuery query, RelNode r) {
+ final ImmutableList.Builder<RelNode> builder = ImmutableList.builder();
+ return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
+ query.druidTable, query.intervals, builder.addAll(query.rels).add(r).build());
+ }
+
+ /** Extends a DruidQuery. */
+ public static DruidQuery extendQuery(DruidQuery query, List<Interval> intervals) {
+ return DruidQuery.create(query.getCluster(), query.getTraitSet(), query.getTable(),
+ query.druidTable, intervals, query.rels);
+ }
+
+ @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+ assert inputs.isEmpty();
+ return this;
+ }
+
+ @Override public RelDataType deriveRowType() {
+ return getCluster().getTypeFactory().createStructType(
+ Pair.right(Util.last(rels).getRowType().getFieldList()),
+ getQuerySpec().fieldNames);
+ }
+
+ public TableScan getTableScan() {
+ return (TableScan) rels.get(0);
+ }
+
+ public RelNode getTopNode() {
+ return Util.last(rels);
+ }
+
+ @Override public RelOptTable getTable() {
+ return table;
+ }
+
+ @Override public RelWriter explainTerms(RelWriter pw) {
+ for (RelNode rel : rels) {
+ if (rel instanceof TableScan) {
+ TableScan tableScan = (TableScan) rel;
+ pw.item("table", tableScan.getTable().getQualifiedName());
+ pw.item("intervals", intervals);
+ } else if (rel instanceof Filter) {
+ pw.item("filter", ((Filter) rel).getCondition());
+ } else if (rel instanceof Project) {
+ pw.item("projects", ((Project) rel).getProjects());
+ } else if (rel instanceof Aggregate) {
+ final Aggregate aggregate = (Aggregate) rel;
+ pw.item("groups", aggregate.getGroupSet())
+ .item("aggs", aggregate.getAggCallList());
+ } else if (rel instanceof Sort) {
+ final Sort sort = (Sort) rel;
+ for (Ord<RelFieldCollation> ord
+ : Ord.zip(sort.collation.getFieldCollations())) {
+ pw.item("sort" + ord.i, ord.e.getFieldIndex());
+ }
+ for (Ord<RelFieldCollation> ord
+ : Ord.zip(sort.collation.getFieldCollations())) {
+ pw.item("dir" + ord.i, ord.e.shortString());
+ }
+ pw.itemIf("fetch", sort.fetch, sort.fetch != null);
+ } else {
+ throw new AssertionError("rel type not supported in Druid query "
+ + rel);
+ }
+ }
+ return pw;
+ }
+
+ @Override public RelOptCost computeSelfCost(RelOptPlanner planner,
+ RelMetadataQuery mq) {
+ // Heuristic: we assume pushing query to Druid reduces cost by 90%
+ return Util.last(rels).computeSelfCost(planner, mq).multiplyBy(.1);
+ }
+
+ @Override public RelNode project(ImmutableBitSet fieldsUsed,
+ Set<RelDataTypeField> extraFields,
+ RelBuilder relBuilder) {
+ final int fieldCount = getRowType().getFieldCount();
+ if (fieldsUsed.equals(ImmutableBitSet.range(fieldCount))
+ && extraFields.isEmpty()) {
+ return this;
+ }
+ final List<RexNode> exprList = new ArrayList<>();
+ final List<String> nameList = new ArrayList<>();
+ final RexBuilder rexBuilder = getCluster().getRexBuilder();
+ final List<RelDataTypeField> fields = getRowType().getFieldList();
+
+ // Project the subset of fields.
+ for (int i : fieldsUsed) {
+ RelDataTypeField field = fields.get(i);
+ exprList.add(rexBuilder.makeInputRef(this, i));
+ nameList.add(field.getName());
+ }
+
+ // Project nulls for the extra fields. (Maybe a sub-class table has
+ // extra fields, but we don't.)
+ for (RelDataTypeField extraField : extraFields) {
+ exprList.add(
+ rexBuilder.ensureType(
+ extraField.getType(),
+ rexBuilder.constantNull(),
+ true));
+ nameList.add(extraField.getName());
+ }
+
+ HiveProject hp = (HiveProject) relBuilder.push(this).project(exprList, nameList).build();
+ hp.setSynthetic();
+ return hp;
+ }
+
+ public QuerySpec getQuerySpec() {
+ if (querySpec == null) {
+ querySpec = deriveQuerySpec();
+ assert querySpec != null : this;
+ }
+ return querySpec;
+ }
+
+ protected QuerySpec deriveQuerySpec() {
+ final RelDataType rowType = table.getRowType();
+ int i = 1;
+
+ RexNode filter = null;
+ if (i < rels.size() && rels.get(i) instanceof Filter) {
+ final Filter filterRel = (Filter) rels.get(i++);
+ filter = filterRel.getCondition();
+ }
+
+ List<RexNode> projects = null;
+ if (i < rels.size() && rels.get(i) instanceof Project) {
+ final Project project = (Project) rels.get(i++);
+ projects = project.getProjects();
+ }
+
+ ImmutableBitSet groupSet = null;
+ List<AggregateCall> aggCalls = null;
+ List<String> aggNames = null;
+ if (i < rels.size() && rels.get(i) instanceof Aggregate) {
+ final Aggregate aggregate = (Aggregate) rels.get(i++);
+ groupSet = aggregate.getGroupSet();
+ aggCalls = aggregate.getAggCallList();
+ aggNames = Util.skip(aggregate.getRowType().getFieldNames(),
+ groupSet.cardinality());
+ }
+
+ List<Integer> collationIndexes = null;
+ List<Direction> collationDirections = null;
+ Integer fetch = null;
+ if (i < rels.size() && rels.get(i) instanceof Sort) {
+ final Sort sort = (Sort) rels.get(i++);
+ collationIndexes = new ArrayList<>();
+ collationDirections = new ArrayList<>();
+ for (RelFieldCollation fCol: sort.collation.getFieldCollations()) {
+ collationIndexes.add(fCol.getFieldIndex());
+ collationDirections.add(fCol.getDirection());
+ }
+ fetch = sort.fetch != null ? RexLiteral.intValue(sort.fetch) : null;
+ }
+
+ if (i != rels.size()) {
+ throw new AssertionError("could not implement all rels");
+ }
+
+ return getQuery(rowType, filter, projects, groupSet, aggCalls, aggNames,
+ collationIndexes, collationDirections, fetch);
+ }
+
+ public String getQueryType() {
+ return getQuerySpec().queryType.getQueryName();
+ }
+
+ public String getQueryString() {
+ return getQuerySpec().queryString;
+ }
+
+ private QuerySpec getQuery(RelDataType rowType, RexNode filter, List<RexNode> projects,
+ ImmutableBitSet groupSet, List<AggregateCall> aggCalls, List<String> aggNames,
+ List<Integer> collationIndexes, List<Direction> collationDirections, Integer fetch) {
+ DruidQueryType queryType = DruidQueryType.SELECT;
+ final Translator translator = new Translator(druidTable, rowType);
+ List<String> fieldNames = rowType.getFieldNames();
+
+ // Handle filter
+ Json jsonFilter = null;
+ if (filter != null) {
+ jsonFilter = translator.translateFilter(filter);
+ }
+
+ // Then we handle project
+ if (projects != null) {
+ translator.metrics.clear();
+ translator.dimensions.clear();
+ final ImmutableList.Builder<String> builder = ImmutableList.builder();
+ for (RexNode project : projects) {
+ builder.add(translator.translate(project, true));
+ }
+ fieldNames = builder.build();
+ }
+
+ // Finally we handle aggregate and sort. Handling of these
+ // operators is more complex, since we need to extract
+ // the conditions to know whether the query will be
+ // executed as a Timeseries, TopN, or GroupBy in Druid
+ final List<String> dimensions = new ArrayList<>();
+ final List<JsonAggregation> aggregations = new ArrayList<>();
+ String granularity = "ALL";
+ Direction timeSeriesDirection = null;
+ JsonLimit limit = null;
+ if (groupSet != null) {
+ assert aggCalls != null;
+ assert aggNames != null;
+ assert aggCalls.size() == aggNames.size();
+
+ int timePositionIdx = -1;
+ final ImmutableList.Builder<String> builder = ImmutableList.builder();
+ if (projects != null) {
+ for (int groupKey : groupSet) {
+ final String s = fieldNames.get(groupKey);
+ final RexNode project = projects.get(groupKey);
+ if (project instanceof RexInputRef) {
+ // Reference, it could be to the timestamp column or any other dimension
+ final RexInputRef ref = (RexInputRef) project;
+ final String origin = druidTable.rowType.getFieldList().get(ref.getIndex()).getName();
+ if (origin.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+ granularity = "NONE";
+ builder.add(s);
+ assert timePositionIdx == -1;
+ timePositionIdx = groupKey;
+ } else {
+ dimensions.add(s);
+ builder.add(s);
+ }
+ } else if (project instanceof RexCall) {
+ // Call, check if we should infer granularity
+ RexCall call = (RexCall) project;
+ if (HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
+ granularity = call.getOperator().getName();
+ builder.add(s);
+ assert timePositionIdx == -1;
+ timePositionIdx = groupKey;
+ } else {
+ dimensions.add(s);
+ builder.add(s);
+ }
+ } else {
+ throw new AssertionError("incompatible project expression: " + project);
+ }
+ }
+ } else {
+ for (int groupKey : groupSet) {
+ final String s = fieldNames.get(groupKey);
+ if (s.equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+ granularity = "NONE";
+ builder.add(s);
+ assert timePositionIdx == -1;
+ timePositionIdx = groupKey;
+ } else {
+ dimensions.add(s);
+ builder.add(s);
+ }
+ }
+ }
+
+ for (Pair<AggregateCall, String> agg : Pair.zip(aggCalls, aggNames)) {
+ final JsonAggregation jsonAggregation =
+ getJsonAggregation(fieldNames, agg.right, agg.left);
+ aggregations.add(jsonAggregation);
+ builder.add(jsonAggregation.name);
+ }
+
+ fieldNames = builder.build();
+
+ ImmutableList<JsonCollation> collations = null;
+ boolean sortsMetric = false;
+ if (collationIndexes != null) {
+ assert collationDirections != null;
+ ImmutableList.Builder<JsonCollation> colBuilder = new ImmutableList.Builder<JsonCollation>();
+ for (Pair<Integer,Direction> p : Pair.zip(collationIndexes, collationDirections)) {
+ colBuilder.add(new JsonCollation(fieldNames.get(p.left),
+ p.right == Direction.DESCENDING ? "descending" : "ascending"));
+ if (p.left >= groupSet.cardinality() && p.right == Direction.DESCENDING) {
+ // Currently only support for DESC in TopN
+ sortsMetric = true;
+ } else if (p.left == timePositionIdx) {
+ assert timeSeriesDirection == null;
+ timeSeriesDirection = p.right;
+ }
+ }
+ collations = colBuilder.build();
+ }
+
+ limit = new JsonLimit("default", fetch, collations);
+
+ if (dimensions.isEmpty() && (collations == null || timeSeriesDirection != null)) {
+ queryType = DruidQueryType.TIMESERIES;
+ assert fetch == null;
+ } else if (dimensions.size() == 1 && sortsMetric && collations.size() == 1 && fetch != null) {
+ queryType = DruidQueryType.TOP_N;
+ } else {
+ queryType = DruidQueryType.GROUP_BY;
+ }
+ } else {
+ assert aggCalls == null;
+ assert aggNames == null;
+ assert collationIndexes == null || collationIndexes.isEmpty();
+ assert collationDirections == null || collationDirections.isEmpty();
+ }
+
+ final StringWriter sw = new StringWriter();
+ final JsonFactory factory = new JsonFactory();
+ try {
+ final JsonGenerator generator = factory.createGenerator(sw);
+
+ switch (queryType) {
+ case TIMESERIES:
+ generator.writeStartObject();
+
+ generator.writeStringField("queryType", "timeseries");
+ generator.writeStringField("dataSource", druidTable.dataSource);
+ generator.writeStringField("descending", timeSeriesDirection != null &&
+ timeSeriesDirection == Direction.DESCENDING ? "true" : "false");
+ generator.writeStringField("granularity", granularity);
+ writeFieldIf(generator, "filter", jsonFilter);
+ writeField(generator, "aggregations", aggregations);
+ writeFieldIf(generator, "postAggregations", null);
+ writeField(generator, "intervals", intervals);
+
+ generator.writeEndObject();
+ break;
+
+ case TOP_N:
+ generator.writeStartObject();
+
+ generator.writeStringField("queryType", "topN");
+ generator.writeStringField("dataSource", druidTable.dataSource);
+ generator.writeStringField("granularity", granularity);
+ generator.writeStringField("dimension", dimensions.get(0));
+ generator.writeStringField("metric", fieldNames.get(collationIndexes.get(0)));
+ writeFieldIf(generator, "filter", jsonFilter);
+ writeField(generator, "aggregations", aggregations);
+ writeFieldIf(generator, "postAggregations", null);
+ writeField(generator, "intervals", intervals);
+ generator.writeNumberField("threshold", fetch);
+
+ generator.writeEndObject();
+ break;
+
+ case GROUP_BY:
+ generator.writeStartObject();
+
+ if (aggregations.isEmpty()) {
+ // Druid requires at least one aggregation, otherwise gives:
+ // Must have at least one AggregatorFactory
+ aggregations.add(
+ new JsonAggregation("longSum", "dummy_agg", "dummy_agg"));
+ }
+
+ generator.writeStringField("queryType", "groupBy");
+ generator.writeStringField("dataSource", druidTable.dataSource);
+ generator.writeStringField("granularity", granularity);
+ writeField(generator, "dimensions", dimensions);
+ writeFieldIf(generator, "limitSpec", limit);
+ writeFieldIf(generator, "filter", jsonFilter);
+ writeField(generator, "aggregations", aggregations);
+ writeFieldIf(generator, "postAggregations", null);
+ writeField(generator, "intervals", intervals);
+ writeFieldIf(generator, "having", null);
+
+ generator.writeEndObject();
+ break;
+
+ case SELECT:
+ generator.writeStartObject();
+
+ generator.writeStringField("queryType", "select");
+ generator.writeStringField("dataSource", druidTable.dataSource);
+ generator.writeStringField("descending", "false");
+ writeField(generator, "intervals", intervals);
+ writeFieldIf(generator, "filter", jsonFilter);
+ writeField(generator, "dimensions", translator.dimensions);
+ writeField(generator, "metrics", translator.metrics);
+ generator.writeStringField("granularity", granularity);
+
+ generator.writeFieldName("pagingSpec");
+ generator.writeStartObject();
+ generator.writeNumberField("threshold", fetch != null ? fetch : 1);
+ generator.writeEndObject();
+
+ generator.writeFieldName("context");
+ generator.writeStartObject();
+ generator.writeBooleanField(Constants.DRUID_QUERY_FETCH, fetch != null);
+ generator.writeEndObject();
+
+ generator.writeEndObject();
+ break;
+
+ default:
+ throw new AssertionError("unknown query type " + queryType);
+ }
+
+ generator.close();
+ } catch (IOException e) {
+ e.printStackTrace();
+ }
+
+ return new QuerySpec(queryType, sw.toString(), fieldNames);
+ }
+
+ private JsonAggregation getJsonAggregation(List<String> fieldNames,
+ String name, AggregateCall aggCall) {
+ final List<String> list = new ArrayList<>();
+ for (Integer arg : aggCall.getArgList()) {
+ list.add(fieldNames.get(arg));
+ }
+ final String only = Iterables.getFirst(list, null);
+ final boolean b = aggCall.getType().getSqlTypeName() == SqlTypeName.DOUBLE;
+ switch (aggCall.getAggregation().getKind()) {
+ case COUNT:
+ if (aggCall.isDistinct()) {
+ return new JsonCardinalityAggregation("cardinality", name, list);
+ }
+ return new JsonAggregation("count", name, only);
+ case SUM:
+ case SUM0:
+ return new JsonAggregation(b ? "doubleSum" : "longSum", name, only);
+ case MIN:
+ return new JsonAggregation(b ? "doubleMin" : "longMin", name, only);
+ case MAX:
+ return new JsonAggregation(b ? "doubleMax" : "longMax", name, only);
+ default:
+ throw new AssertionError("unknown aggregate " + aggCall);
+ }
+ }
+
+ private static void writeField(JsonGenerator generator, String fieldName,
+ Object o) throws IOException {
+ generator.writeFieldName(fieldName);
+ writeObject(generator, o);
+ }
+
+ private static void writeFieldIf(JsonGenerator generator, String fieldName,
+ Object o) throws IOException {
+ if (o != null) {
+ writeField(generator, fieldName, o);
+ }
+ }
+
+ private static void writeArray(JsonGenerator generator, List<?> elements)
+ throws IOException {
+ generator.writeStartArray();
+ for (Object o : elements) {
+ writeObject(generator, o);
+ }
+ generator.writeEndArray();
+ }
+
+ private static void writeObject(JsonGenerator generator, Object o)
+ throws IOException {
+ if (o instanceof String) {
+ String s = (String) o;
+ generator.writeString(s);
+ } else if (o instanceof Interval) {
+ Interval i = (Interval) o;
+ generator.writeString(i.toString());
+ } else if (o instanceof Integer) {
+ Integer i = (Integer) o;
+ generator.writeNumber(i);
+ } else if (o instanceof List) {
+ writeArray(generator, (List<?>) o);
+ } else if (o instanceof Json) {
+ ((Json) o).write(generator);
+ } else {
+ throw new AssertionError("not a json object: " + o);
+ }
+ }
+
+ /** Druid query specification. */
+ public static class QuerySpec {
+ final DruidQueryType queryType;
+ final String queryString;
+ final List<String> fieldNames;
+
+ QuerySpec(DruidQueryType queryType, String queryString,
+ List<String> fieldNames) {
+ this.queryType = Preconditions.checkNotNull(queryType);
+ this.queryString = Preconditions.checkNotNull(queryString);
+ this.fieldNames = ImmutableList.copyOf(fieldNames);
+ }
+
+ @Override public int hashCode() {
+ return Objects.hash(queryType, queryString, fieldNames);
+ }
+
+ @Override public boolean equals(Object obj) {
+ return obj == this
+ || obj instanceof QuerySpec
+ && queryType == ((QuerySpec) obj).queryType
+ && queryString.equals(((QuerySpec) obj).queryString)
+ && fieldNames.equals(((QuerySpec) obj).fieldNames);
+ }
+
+ @Override public String toString() {
+ return "{queryType: " + queryType
+ + ", queryString: " + queryString
+ + ", fieldNames: " + fieldNames + "}";
+ }
+
+ String getQueryString(String pagingIdentifier, int offset) {
+ if (pagingIdentifier == null) {
+ return queryString;
+ }
+ return queryString.replace("\"threshold\":",
+ "\"pagingIdentifiers\":{\"" + pagingIdentifier + "\":" + offset
+ + "},\"threshold\":");
+ }
+ }
+
+ /** Translates scalar expressions to Druid field references. */
+ private static class Translator {
+ final List<String> dimensions = new ArrayList<>();
+ final List<String> metrics = new ArrayList<>();
+ final DruidTable druidTable;
+ final RelDataType rowType;
+
+ Translator(DruidTable druidTable, RelDataType rowType) {
+ this.druidTable = druidTable;
+ this.rowType = rowType;
+ for (RelDataTypeField f : rowType.getFieldList()) {
+ final String fieldName = f.getName();
+ if (druidTable.metricFieldNames.contains(fieldName)) {
+ metrics.add(fieldName);
+ } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
+ dimensions.add(fieldName);
+ }
+ }
+ }
+
+ String translate(RexNode e, boolean set) {
+ switch (e.getKind()) {
+ case INPUT_REF:
+ final RexInputRef ref = (RexInputRef) e;
+ final String fieldName =
+ rowType.getFieldList().get(ref.getIndex()).getName();
+ if (set) {
+ if (druidTable.metricFieldNames.contains(fieldName)) {
+ metrics.add(fieldName);
+ } else if (!DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(fieldName)) {
+ dimensions.add(fieldName);
+ }
+ }
+ return fieldName;
+
+ case CAST:
+ return tr(e, 0, set);
+
+ case LITERAL:
+ return ((RexLiteral) e).getValue2().toString();
+
+ case OTHER_FUNCTION:
+ final RexCall call = (RexCall) e;
+ assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
+ return tr(call, 0, set);
+
+ default:
+ throw new AssertionError("invalid expression " + e);
+ }
+ }
+
+ @SuppressWarnings("incomplete-switch")
+ private JsonFilter translateFilter(RexNode e) {
+ RexCall call;
+ switch (e.getKind()) {
+ case EQUALS:
+ case NOT_EQUALS:
+ case GREATER_THAN:
+ case GREATER_THAN_OR_EQUAL:
+ case LESS_THAN:
+ case LESS_THAN_OR_EQUAL:
+ call = (RexCall) e;
+ int posRef;
+ int posConstant;
+ if (RexUtil.isConstant(call.getOperands().get(1))) {
+ posRef = 0;
+ posConstant = 1;
+ } else if (RexUtil.isConstant(call.getOperands().get(0))) {
+ posRef = 1;
+ posConstant = 0;
+ } else {
+ throw new AssertionError("it is not a valid comparison: " + e);
+ }
+ switch (e.getKind()) {
+ case EQUALS:
+ return new JsonSelector("selector", tr(e, posRef), tr(e, posConstant));
+ case NOT_EQUALS:
+ return new JsonCompositeFilter("not",
+ ImmutableList.of(new JsonSelector("selector", tr(e, posRef), tr(e, posConstant))));
+ case GREATER_THAN:
+ return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), true, null, false,
+ false);
+ case GREATER_THAN_OR_EQUAL:
+ return new JsonBound("bound", tr(e, posRef), tr(e, posConstant), false, null, false,
+ false);
+ case LESS_THAN:
+ return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), true,
+ false);
+ case LESS_THAN_OR_EQUAL:
+ return new JsonBound("bound", tr(e, posRef), null, false, tr(e, posConstant), false,
+ false);
+ }
+ case AND:
+ case OR:
+ case NOT:
+ call = (RexCall) e;
+ return new JsonCompositeFilter(e.getKind().toString().toLowerCase(),
+ translateFilters(call.getOperands()));
+ default:
+ throw new AssertionError("cannot translate filter: " + e);
+ }
+ }
+
+ private String tr(RexNode call, int index) {
+ return tr(call, index, false);
+ }
+
+ private String tr(RexNode call, int index, boolean set) {
+ return translate(((RexCall) call).getOperands().get(index), set);
+ }
+
+ private List<JsonFilter> translateFilters(List<RexNode> operands) {
+ final ImmutableList.Builder<JsonFilter> builder =
+ ImmutableList.builder();
+ for (RexNode operand : operands) {
+ builder.add(translateFilter(operand));
+ }
+ return builder.build();
+ }
+ }
+
+ /** Object that knows how to write itself to a
+ * {@link com.fasterxml.jackson.core.JsonGenerator}. */
+ private interface Json {
+ void write(JsonGenerator generator) throws IOException;
+ }
+
+ /** Aggregation element of a Druid "groupBy" or "topN" query. */
+ private static class JsonAggregation implements Json {
+ final String type;
+ final String name;
+ final String fieldName;
+
+ private JsonAggregation(String type, String name, String fieldName) {
+ this.type = type;
+ this.name = name;
+ this.fieldName = fieldName;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ generator.writeStringField("name", name);
+ writeFieldIf(generator, "fieldName", fieldName);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Collation element of a Druid "groupBy" query. */
+ private static class JsonLimit implements Json {
+ final String type;
+ final Integer limit;
+ final ImmutableList<JsonCollation> collations;
+
+ private JsonLimit(String type, Integer limit, ImmutableList<JsonCollation> collations) {
+ this.type = type;
+ this.limit = limit;
+ this.collations = collations;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ writeFieldIf(generator, "limit", limit);
+ writeFieldIf(generator, "columns", collations);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Collation element of a Druid "groupBy" query. */
+ private static class JsonCollation implements Json {
+ final String dimension;
+ final String direction;
+
+ private JsonCollation(String dimension, String direction) {
+ this.dimension = dimension;
+ this.direction = direction;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("dimension", dimension);
+ writeFieldIf(generator, "direction", direction);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Aggregation element that calls the "cardinality" function. */
+ private static class JsonCardinalityAggregation extends JsonAggregation {
+ final List<String> fieldNames;
+
+ private JsonCardinalityAggregation(String type, String name,
+ List<String> fieldNames) {
+ super(type, name, null);
+ this.fieldNames = fieldNames;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ generator.writeStringField("name", name);
+ writeFieldIf(generator, "fieldNames", fieldNames);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Filter element of a Druid "groupBy" or "topN" query. */
+ private abstract static class JsonFilter implements Json {
+ final String type;
+
+ private JsonFilter(String type) {
+ this.type = type;
+ }
+ }
+
+ /** Equality filter. */
+ private static class JsonSelector extends JsonFilter {
+ private final String dimension;
+ private final String value;
+
+ private JsonSelector(String type, String dimension, String value) {
+ super(type);
+ this.dimension = dimension;
+ this.value = value;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ generator.writeStringField("dimension", dimension);
+ generator.writeStringField("value", value);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Bound filter. */
+ private static class JsonBound extends JsonFilter {
+ private final String dimension;
+ private final String lower;
+ private final boolean lowerStrict;
+ private final String upper;
+ private final boolean upperStrict;
+ private final boolean alphaNumeric;
+
+ private JsonBound(String type, String dimension, String lower,
+ boolean lowerStrict, String upper, boolean upperStrict,
+ boolean alphaNumeric) {
+ super(type);
+ this.dimension = dimension;
+ this.lower = lower;
+ this.lowerStrict = lowerStrict;
+ this.upper = upper;
+ this.upperStrict = upperStrict;
+ this.alphaNumeric = alphaNumeric;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ generator.writeStringField("dimension", dimension);
+ if (lower != null) {
+ generator.writeStringField("lower", lower);
+ generator.writeBooleanField("lowerStrict", lowerStrict);
+ }
+ if (upper != null) {
+ generator.writeStringField("upper", upper);
+ generator.writeBooleanField("upperStrict", upperStrict);
+ }
+ generator.writeBooleanField("alphaNumeric", alphaNumeric);
+ generator.writeEndObject();
+ }
+ }
+
+ /** Filter that combines other filters using a boolean operator. */
+ private static class JsonCompositeFilter extends JsonFilter {
+ private final List<? extends JsonFilter> fields;
+
+ private JsonCompositeFilter(String type,
+ List<? extends JsonFilter> fields) {
+ super(type);
+ this.fields = fields;
+ }
+
+ public void write(JsonGenerator generator) throws IOException {
+ generator.writeStartObject();
+ generator.writeStringField("type", type);
+ switch (type) {
+ case "NOT":
+ writeField(generator, "field", fields.get(0));
+ break;
+ default:
+ writeField(generator, "fields", fields);
+ }
+ generator.writeEndObject();
+ }
+ }
+
+}
+
+// End DruidQuery.java
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
new file mode 100644
index 0000000..228b307
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidQueryType.java
@@ -0,0 +1,42 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+/**
+ * Type of Druid query.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public enum DruidQueryType {
+ SELECT("select"),
+ TOP_N("topN"),
+ GROUP_BY("groupBy"),
+ TIMESERIES("timeseries");
+
+ private final String queryName;
+
+ private DruidQueryType(String queryName) {
+ this.queryName = queryName;
+ }
+
+ public String getQueryName() {
+ return this.queryName;
+ }
+}
+
+// End QueryType.java
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
new file mode 100644
index 0000000..f68ffa5
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidRules.java
@@ -0,0 +1,591 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.hadoop.hive.ql.optimizer.calcite.HiveCalciteUtil;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveProjectSortTransposeRule;
+import org.apache.hadoop.hive.ql.optimizer.calcite.rules.HiveSortProjectTransposeRule;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * Rules and relational operators for {@link DruidQuery}.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidRules {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(DruidRules.class);
+
+ // Avoid instantiation
+ private DruidRules() {
+ }
+
+ public static final DruidFilterRule FILTER = new DruidFilterRule();
+ public static final DruidProjectRule PROJECT = new DruidProjectRule();
+ public static final DruidAggregateRule AGGREGATE = new DruidAggregateRule();
+ public static final DruidProjectAggregateRule PROJECT_AGGREGATE = new DruidProjectAggregateRule();
+ public static final DruidSortRule SORT = new DruidSortRule();
+ public static final DruidProjectSortRule PROJECT_SORT = new DruidProjectSortRule();
+ public static final DruidSortProjectRule SORT_PROJECT = new DruidSortProjectRule();
+
+ /** Predicate that returns whether Druid can not handle an aggregate. */
+ private static final Predicate<AggregateCall> BAD_AGG = new Predicate<AggregateCall>() {
+ public boolean apply(AggregateCall aggregateCall) {
+ switch (aggregateCall.getAggregation().getKind()) {
+ case COUNT:
+ case SUM:
+ case SUM0:
+ case MIN:
+ case MAX:
+ return false;
+ default:
+ return true;
+ }
+ }
+ };
+
+ /**
+ * Rule to push a {@link org.apache.calcite.rel.core.Filter} into a {@link DruidQuery}.
+ */
+ private static class DruidFilterRule extends RelOptRule {
+ private DruidFilterRule() {
+ super(operand(Filter.class,
+ operand(DruidQuery.class, none())));
+ }
+
+ public void onMatch(RelOptRuleCall call) {
+ final Filter filter = call.rel(0);
+ final DruidQuery query = call.rel(1);
+ if (!DruidQuery.isValidSignature(query.signature() + 'f')
+ || !query.isValidFilter(filter.getCondition())) {
+ return;
+ }
+ // Timestamp
+ int timestampFieldIdx = -1;
+ for (int i = 0; i < query.getRowType().getFieldCount(); i++) {
+ if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(
+ query.getRowType().getFieldList().get(i).getName())) {
+ timestampFieldIdx = i;
+ break;
+ }
+ }
+ final Pair<List<RexNode>, List<RexNode>> pair = splitFilters(
+ filter.getCluster().getRexBuilder(), query, filter.getCondition(), timestampFieldIdx);
+ if (pair == null) {
+ // We can't push anything useful to Druid.
+ return;
+ }
+ List<Interval> intervals = null;
+ if (!pair.left.isEmpty()) {
+ intervals = DruidIntervalUtils.createInterval(
+ query.getRowType().getFieldList().get(timestampFieldIdx).getType(),
+ pair.left);
+ if (intervals == null) {
+ // We can't push anything useful to Druid.
+ return;
+ }
+ }
+ DruidQuery newDruidQuery = query;
+ if (!pair.right.isEmpty()) {
+ if (!validConditions(pair.right)) {
+ return;
+ }
+ final RelNode newFilter = filter.copy(filter.getTraitSet(), Util.last(query.rels),
+ RexUtil.composeConjunction(filter.getCluster().getRexBuilder(), pair.right, false));
+ newDruidQuery = DruidQuery.extendQuery(query, newFilter);
+ }
+ if (intervals != null) {
+ newDruidQuery = DruidQuery.extendQuery(newDruidQuery, intervals);
+ }
+ call.transformTo(newDruidQuery);
+ }
+
+ /* Splits the filter condition in two groups: those that filter on the timestamp column
+ * and those that filter on other fields */
+ private static Pair<List<RexNode>, List<RexNode>> splitFilters(final RexBuilder rexBuilder,
+ final DruidQuery input, RexNode cond, final int timestampFieldIdx) {
+ final List<RexNode> timeRangeNodes = new ArrayList<>();
+ final List<RexNode> otherNodes = new ArrayList<>();
+ List<RexNode> conjs = RelOptUtil.conjunctions(cond);
+ if (conjs.isEmpty()) {
+ // We do not transform
+ return null;
+ }
+ // Number of columns with the dimensions and timestamp
+ int max = input.getRowType().getFieldCount() - input.druidTable.metricFieldNames.size();
+ for (RexNode conj : conjs) {
+ final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
+ conj.accept(visitor);
+ if (visitor.inputPosReferenced.contains(timestampFieldIdx)) {
+ if (visitor.inputPosReferenced.size() != 1) {
+ // Complex predicate, transformation currently not supported
+ return null;
+ }
+ timeRangeNodes.add(conj);
+ } else if (!visitor.inputPosReferenced.tailSet(max).isEmpty()) {
+ // Filter on metrics, not supported in Druid
+ return null;
+ } else {
+ otherNodes.add(conj);
+ }
+ }
+ return Pair.of(timeRangeNodes, otherNodes);
+ }
+
+ /* Checks that all conditions are on ref + literal*/
+ private static boolean validConditions(List<RexNode> nodes) {
+ for (RexNode node: nodes) {
+ try {
+ node.accept(
+ new RexVisitorImpl<Void>(true) {
+ @SuppressWarnings("incomplete-switch")
+ @Override public Void visitCall(RexCall call) {
+ switch (call.getKind()) {
+ case CAST:
+ // Only if on top of ref or literal
+ if (call.getOperands().get(0) instanceof RexInputRef ||
+ call.getOperands().get(0) instanceof RexLiteral) {
+ break;
+ }
+ // Not supported
+ throw Util.FoundOne.NULL;
+ case EQUALS:
+ case LESS_THAN:
+ case LESS_THAN_OR_EQUAL:
+ case GREATER_THAN:
+ case GREATER_THAN_OR_EQUAL:
+ // Check cast
+ RexNode left = call.getOperands().get(0);
+ if (left.getKind() == SqlKind.CAST) {
+ left = ((RexCall)left).getOperands().get(0);
+ }
+ RexNode right = call.getOperands().get(1);
+ if (right.getKind() == SqlKind.CAST) {
+ right = ((RexCall)right).getOperands().get(0);
+ }
+ if (left instanceof RexInputRef && right instanceof RexLiteral) {
+ break;
+ }
+ if (right instanceof RexInputRef && left instanceof RexLiteral) {
+ break;
+ }
+ // Not supported if it is not ref + literal
+ throw Util.FoundOne.NULL;
+ case BETWEEN:
+ case IN:
+ // Not supported here yet
+ throw Util.FoundOne.NULL;
+ }
+ return super.visitCall(call);
+ }
+ });
+ } catch (Util.FoundOne e) {
+ return false;
+ }
+ }
+ return true;
+ }
+ }
+
+ /**
+ * Rule to push a {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
+ */
+ private static class DruidProjectRule extends RelOptRule {
+ private DruidProjectRule() {
+ super(operand(Project.class,
+ operand(DruidQuery.class, none())));
+ }
+
+ public void onMatch(RelOptRuleCall call) {
+ final Project project = call.rel(0);
+ final DruidQuery query = call.rel(1);
+ if (!DruidQuery.isValidSignature(query.signature() + 'p')) {
+ return;
+ }
+
+ if (canProjectAll(project.getProjects())) {
+ // All expressions can be pushed to Druid in their entirety.
+ final RelNode newProject = project.copy(project.getTraitSet(),
+ ImmutableList.of(Util.last(query.rels)));
+ RelNode newNode = DruidQuery.extendQuery(query, newProject);
+ call.transformTo(newNode);
+ return;
+ }
+ final Pair<List<RexNode>, List<RexNode>> pair = splitProjects(
+ project.getCluster().getRexBuilder(), query, project.getProjects());
+ if (pair == null) {
+ // We can't push anything useful to Druid.
+ return;
+ }
+ final List<RexNode> above = pair.left;
+ final List<RexNode> below = pair.right;
+ final RelDataTypeFactory.FieldInfoBuilder builder = project.getCluster().getTypeFactory()
+ .builder();
+ final RelNode input = Util.last(query.rels);
+ for (RexNode e : below) {
+ final String name;
+ if (e instanceof RexInputRef) {
+ name = input.getRowType().getFieldNames().get(((RexInputRef) e).getIndex());
+ } else {
+ name = null;
+ }
+ builder.add(name, e.getType());
+ }
+ final RelNode newProject = project.copy(project.getTraitSet(), input, below, builder.build());
+ final DruidQuery newQuery = DruidQuery.extendQuery(query, newProject);
+ final RelNode newProject2 = project.copy(project.getTraitSet(), newQuery, above,
+ project.getRowType());
+ call.transformTo(newProject2);
+ }
+
+ private static boolean canProjectAll(List<RexNode> nodes) {
+ for (RexNode e : nodes) {
+ if (!(e instanceof RexInputRef)) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ private static Pair<List<RexNode>, List<RexNode>> splitProjects(final RexBuilder rexBuilder,
+ final RelNode input, List<RexNode> nodes) {
+ final RelOptUtil.InputReferencedVisitor visitor = new RelOptUtil.InputReferencedVisitor();
+ for (RexNode node : nodes) {
+ node.accept(visitor);
+ }
+ if (visitor.inputPosReferenced.size() == input.getRowType().getFieldCount()) {
+ // All inputs are referenced
+ return null;
+ }
+ final List<RexNode> belowNodes = new ArrayList<>();
+ final List<RelDataType> belowTypes = new ArrayList<>();
+ final List<Integer> positions = Lists.newArrayList(visitor.inputPosReferenced);
+ for (int i : positions) {
+ final RexNode node = rexBuilder.makeInputRef(input, i);
+ belowNodes.add(node);
+ belowTypes.add(node.getType());
+ }
+ final List<RexNode> aboveNodes = new ArrayList<>();
+ for (RexNode node : nodes) {
+ aboveNodes.add(node.accept(new RexShuttle() {
+ @Override
+ public RexNode visitInputRef(RexInputRef ref) {
+ final int index = positions.indexOf(ref.getIndex());
+ return rexBuilder.makeInputRef(belowTypes.get(index), index);
+ }
+ }));
+ }
+ return Pair.of(aboveNodes, belowNodes);
+ }
+ }
+
+ /**
+ * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
+ */
+ private static class DruidAggregateRule extends RelOptRule {
+ private DruidAggregateRule() {
+ super(operand(Aggregate.class,
+ operand(DruidQuery.class, none())));
+ }
+
+ public void onMatch(RelOptRuleCall call) {
+ final Aggregate aggregate = call.rel(0);
+ final DruidQuery query = call.rel(1);
+ if (!DruidQuery.isValidSignature(query.signature() + 'a')) {
+ return;
+ }
+ if (aggregate.indicator
+ || aggregate.getGroupSets().size() != 1
+ || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
+ || !validAggregate(aggregate, query)) {
+ return;
+ }
+ final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
+ ImmutableList.of(Util.last(query.rels)));
+ call.transformTo(DruidQuery.extendQuery(query, newAggregate));
+ }
+
+ /* Check whether agg functions reference timestamp */
+ private static boolean validAggregate(Aggregate aggregate, DruidQuery query) {
+ ImmutableBitSet.Builder builder = ImmutableBitSet.builder();
+ for (AggregateCall aggCall : aggregate.getAggCallList()) {
+ builder.addAll(aggCall.getArgList());
+ }
+ return !checkTimestampRefOnQuery(builder.build(), query.getTopNode());
+ }
+ }
+
+ /**
+ * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} and
+ * {@link org.apache.calcite.rel.core.Project} into a {@link DruidQuery}.
+ */
+ private static class DruidProjectAggregateRule extends RelOptRule {
+ private DruidProjectAggregateRule() {
+ super(operand(Aggregate.class,
+ operand(Project.class,
+ operand(DruidQuery.class, none()))));
+ }
+
+ public void onMatch(RelOptRuleCall call) {
+ final Aggregate aggregate = call.rel(0);
+ final Project project = call.rel(1);
+ final DruidQuery query = call.rel(2);
+ if (!DruidQuery.isValidSignature(query.signature() + 'p' + 'a')) {
+ return;
+ }
+ int timestampIdx;
+ if ((timestampIdx = validProject(project, query)) == -1) {
+ return;
+ }
+ if (aggregate.indicator
+ || aggregate.getGroupSets().size() != 1
+ || Iterables.any(aggregate.getAggCallList(), BAD_AGG)
+ || !validAggregate(aggregate, timestampIdx)) {
+ return;
+ }
+
+ final RelNode newProject = project.copy(project.getTraitSet(),
+ ImmutableList.of(Util.last(query.rels)));
+ final DruidQuery projectDruidQuery = DruidQuery.extendQuery(query, newProject);
+ final RelNode newAggregate = aggregate.copy(aggregate.getTraitSet(),
+ ImmutableList.of(Util.last(projectDruidQuery.rels)));
+ call.transformTo(DruidQuery.extendQuery(projectDruidQuery, newAggregate));
+ }
+
+ /* To be a valid Project, we allow it to contain references, and a single call
+ * to an EXTRACT function on the timestamp column. Returns the reference to
+ * the timestamp, if any. */
+ private static int validProject(Project project, DruidQuery query) {
+ List<RexNode> nodes = project.getProjects();
+ int idxTimestamp = -1;
+ for (int i = 0; i < nodes.size(); i++) {
+ final RexNode e = nodes.get(i);
+ if (e instanceof RexCall) {
+ // It is a call, check that it is EXTRACT and follow-up conditions
+ final RexCall call = (RexCall) e;
+ if (!HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator())) {
+ return -1;
+ }
+ if (idxTimestamp != -1) {
+ // Already one usage of timestamp column
+ return -1;
+ }
+ if (!(call.getOperands().get(0) instanceof RexInputRef)) {
+ return -1;
+ }
+ final RexInputRef ref = (RexInputRef) call.getOperands().get(0);
+ if (!(checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode()))) {
+ return -1;
+ }
+ idxTimestamp = i;
+ continue;
+ }
+ if (!(e instanceof RexInputRef)) {
+ // It needs to be a reference
+ return -1;
+ }
+ final RexInputRef ref = (RexInputRef) e;
+ if (checkTimestampRefOnQuery(ImmutableBitSet.of(ref.getIndex()), query.getTopNode())) {
+ if (idxTimestamp != -1) {
+ // Already one usage of timestamp column
+ return -1;
+ }
+ idxTimestamp = i;
+ }
+ }
+ return idxTimestamp;
+ }
+
+ private static boolean validAggregate(Aggregate aggregate, int idx) {
+ if (!aggregate.getGroupSet().get(idx)) {
+ return false;
+ }
+ for (AggregateCall aggCall : aggregate.getAggCallList()) {
+ if (aggCall.getArgList().contains(idx)) {
+ return false;
+ }
+ }
+ return true;
+ }
+ }
+
+ /**
+ * Rule to push an {@link org.apache.calcite.rel.core.Sort} through a
+ * {@link org.apache.calcite.rel.core.Project}. Useful to transform
+ * to complex Druid queries.
+ */
+ private static class DruidProjectSortRule extends HiveSortProjectTransposeRule {
+ private DruidProjectSortRule() {
+ super(operand(Sort.class,
+ operand(Project.class,
+ operand(DruidQuery.class, none()))));
+ }
+
+ @Override
+ public boolean matches(RelOptRuleCall call) {
+ return true;
+ }
+
+ }
+
+ /**
+ * Rule to push back {@link org.apache.calcite.rel.core.Project} through a
+ * {@link org.apache.calcite.rel.core.Sort}. Useful if after pushing Sort,
+ * we could not push it inside DruidQuery.
+ */
+ private static class DruidSortProjectRule extends HiveProjectSortTransposeRule {
+ private DruidSortProjectRule() {
+ super(operand(Project.class,
+ operand(Sort.class,
+ operand(DruidQuery.class, none()))));
+ }
+ }
+
+ /**
+ * Rule to push an {@link org.apache.calcite.rel.core.Aggregate} into a {@link DruidQuery}.
+ */
+ private static class DruidSortRule extends RelOptRule {
+ private DruidSortRule() {
+ super(operand(Sort.class,
+ operand(DruidQuery.class, none())));
+ }
+
+ public void onMatch(RelOptRuleCall call) {
+ final Sort sort = call.rel(0);
+ final DruidQuery query = call.rel(1);
+ if (!DruidQuery.isValidSignature(query.signature() + 'l')) {
+ return;
+ }
+ // Either it is:
+ // - a sort without limit on the time column on top of
+ // Agg operator (transformable to timeseries query), or
+ // - it is a sort w/o limit on columns that do not include
+ // the time column on top of Agg operator, or
+ // - a simple limit on top of other operator than Agg
+ if (!validSortLimit(sort, query)) {
+ return;
+ }
+ final RelNode newSort = sort.copy(sort.getTraitSet(),
+ ImmutableList.of(Util.last(query.rels)));
+ call.transformTo(DruidQuery.extendQuery(query, newSort));
+ }
+
+ /* Check sort valid */
+ private static boolean validSortLimit(Sort sort, DruidQuery query) {
+ if (sort.offset != null && RexLiteral.intValue(sort.offset) != 0) {
+ // offset not supported by Druid
+ return false;
+ }
+ if (query.getTopNode() instanceof Aggregate) {
+ final Aggregate topAgg = (Aggregate) query.getTopNode();
+ final ImmutableBitSet.Builder positionsReferenced = ImmutableBitSet.builder();
+ int metricsRefs = 0;
+ for (RelFieldCollation col : sort.collation.getFieldCollations()) {
+ int idx = col.getFieldIndex();
+ if (idx >= topAgg.getGroupCount()) {
+ metricsRefs++;
+ continue;
+ }
+ positionsReferenced.set(topAgg.getGroupSet().nth(idx));
+ }
+ boolean refsTimestamp =
+ checkTimestampRefOnQuery(positionsReferenced.build(), topAgg.getInput());
+ if (refsTimestamp && metricsRefs != 0) {
+ return false;
+ }
+ return true;
+ }
+ // If it is going to be a Druid select operator, we push the limit iff
+ // 1) it does not contain a sort specification (required by Druid) and
+ // 2) limit is smaller than select threshold, as otherwise it might be
+ // better to obtain some parallelization and let global limit
+ // optimizer kick in
+ HiveDruidConf conf = sort.getCluster().getPlanner()
+ .getContext().unwrap(HiveDruidConf.class);
+ return HiveCalciteUtil.pureLimitRelNode(sort) &&
+ RexLiteral.intValue(sort.fetch) <= conf.getSelectThreshold();
+ }
+ }
+
+ /* Check if any of the references leads to the timestamp column */
+ private static boolean checkTimestampRefOnQuery(ImmutableBitSet set, RelNode top) {
+ if (top instanceof Project) {
+ ImmutableBitSet.Builder newSet = ImmutableBitSet.builder();
+ final Project project = (Project) top;
+ for (int index : set) {
+ RexNode node = project.getProjects().get(index);
+ if (node instanceof RexInputRef) {
+ newSet.set(((RexInputRef)node).getIndex());
+ } else if (node instanceof RexCall) {
+ RexCall call = (RexCall) node;
+ assert HiveDateGranularity.ALL_FUNCTIONS.contains(call.getOperator());
+ newSet.set(((RexInputRef)call.getOperands().get(0)).getIndex());
+ }
+ }
+ top = project.getInput();
+ set = newSet.build();
+ }
+
+ // Check if any references the timestamp column
+ for (int index : set) {
+ if (DruidTable.DEFAULT_TIMESTAMP_COLUMN.equals(top.getRowType().getFieldNames().get(index))) {
+ return true;
+ }
+ }
+
+ return false;
+ }
+
+}
+
+// End DruidRules.java
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
new file mode 100644
index 0000000..3b3f68a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidSchema.java
@@ -0,0 +1,51 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+import java.util.Map;
+
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.impl.AbstractSchema;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * Schema mapped onto a Druid instance.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidSchema extends AbstractSchema {
+ final String url;
+
+ /**
+ * Creates a Druid schema.
+ *
+ * @param url URL of query REST service
+ */
+ public DruidSchema(String url) {
+ this.url = Preconditions.checkNotNull(url);
+ }
+
+ @Override protected Map<String, Table> getTableMap() {
+ final ImmutableMap.Builder<String, Table> builder = ImmutableMap.builder();
+ return builder.build();
+ }
+}
+
+// End DruidSchema.java
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
new file mode 100644
index 0000000..7288291
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/DruidTable.java
@@ -0,0 +1,121 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.calcite.interpreter.BindableConvention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.impl.AbstractTable;
+import org.joda.time.DateTime;
+import org.joda.time.Interval;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Table mapped onto a Druid table.
+ *
+ * TODO: to be removed when Calcite is upgraded to 1.9
+ */
+public class DruidTable extends AbstractTable implements TranslatableTable {
+
+ public static final String DEFAULT_TIMESTAMP_COLUMN = "__time";
+ public static final Interval DEFAULT_INTERVAL = new Interval(
+ new DateTime("1900-01-01"),
+ new DateTime("3000-01-01")
+ );
+
+ final DruidSchema schema;
+ final String dataSource;
+ final RelDataType rowType;
+ final RelProtoDataType protoRowType;
+ final ImmutableSet<String> metricFieldNames;
+ final ImmutableList<Interval> intervals;
+ final String timestampFieldName;
+
+ /**
+ * Creates a Druid table.
+ *
+ * @param schema Druid schema that contains this table
+ * @param dataSource Druid data source name
+ * @param protoRowType Field names and types
+ * @param metricFieldNames Names of fields that are metrics
+ * @param interval Default interval if query does not constrain the time
+ * @param timestampFieldName Name of the column that contains the time
+ */
+ public DruidTable(DruidSchema schema, String dataSource,
+ RelProtoDataType protoRowType, Set<String> metricFieldNames,
+ List<Interval> intervals, String timestampFieldName) {
+ this.schema = Preconditions.checkNotNull(schema);
+ this.dataSource = Preconditions.checkNotNull(dataSource);
+ this.rowType = null;
+ this.protoRowType = protoRowType;
+ this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
+ this.intervals = ImmutableList.copyOf(intervals);
+ this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
+ }
+
+ public DruidTable(DruidSchema schema, String dataSource,
+ RelDataType rowType, Set<String> metricFieldNames,
+ List<Interval> intervals, String timestampFieldName) {
+ this.schema = Preconditions.checkNotNull(schema);
+ this.dataSource = Preconditions.checkNotNull(dataSource);
+ this.rowType = Preconditions.checkNotNull(rowType);
+ this.protoRowType = null;
+ this.metricFieldNames = ImmutableSet.copyOf(metricFieldNames);
+ this.intervals = ImmutableList.copyOf(intervals);
+ this.timestampFieldName = Preconditions.checkNotNull(timestampFieldName);
+ }
+
+ public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+ final RelDataType thisRowType;
+ if (rowType != null) {
+ thisRowType = rowType;
+ } else {
+ // Generate
+ thisRowType = protoRowType.apply(typeFactory);
+ }
+ final List<String> fieldNames = thisRowType.getFieldNames();
+ Preconditions.checkArgument(fieldNames.contains(timestampFieldName));
+ Preconditions.checkArgument(fieldNames.containsAll(metricFieldNames));
+ return thisRowType;
+ }
+
+ public RelNode toRel(RelOptTable.ToRelContext context,
+ RelOptTable relOptTable) {
+ final RelOptCluster cluster = context.getCluster();
+ final TableScan scan = LogicalTableScan.create(cluster, relOptTable);
+ return DruidQuery.create(cluster,
+ cluster.traitSetOf(BindableConvention.INSTANCE), relOptTable, this,
+ ImmutableList.<RelNode>of(scan));
+ }
+
+}
+
+// End DruidTable.java
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
new file mode 100644
index 0000000..0686dff
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/druid/HiveDruidConf.java
@@ -0,0 +1,33 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.druid;
+
+public class HiveDruidConf {
+
+ private int selectThreshold;
+
+
+ public HiveDruidConf(int selectThreshold) {
+ this.selectThreshold = selectThreshold;
+ }
+
+ public int getSelectThreshold() {
+ return selectThreshold;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
index bc48707..75b7ad2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlCountAggFunction.java
@@ -41,7 +41,7 @@ public class HiveSqlCountAggFunction extends SqlAggFunction implements CanAggreg
SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker) {
super(
"count",
- SqlKind.OTHER_FUNCTION,
+ SqlKind.COUNT,
returnTypeInference,
operandTypeInference,
operandTypeChecker,
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
index 77dca1f..834fc3e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlMinMaxAggFunction.java
@@ -32,7 +32,7 @@ public class HiveSqlMinMaxAggFunction extends SqlAggFunction {
SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker, boolean isMin) {
super(
isMin ? "min" : "max",
- SqlKind.OTHER_FUNCTION,
+ isMin ? SqlKind.MIN : SqlKind.MAX,
returnTypeInference,
operandTypeInference,
operandTypeChecker,
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
index dc286a2..1d551a6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/functions/HiveSqlSumAggFunction.java
@@ -58,7 +58,7 @@ public class HiveSqlSumAggFunction extends SqlAggFunction implements CanAggregat
SqlOperandTypeInference operandTypeInference, SqlOperandTypeChecker operandTypeChecker) {
super(
"sum",
- SqlKind.OTHER_FUNCTION,
+ SqlKind.SUM,
returnTypeInference,
operandTypeInference,
operandTypeChecker,
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
new file mode 100644
index 0000000..b3f8d9b
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/reloperators/HiveDateGranularity.java
@@ -0,0 +1,54 @@
+/**
+ * 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.hadoop.hive.ql.optimizer.calcite.reloperators;
+
+import java.util.Set;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+
+import com.google.common.collect.Sets;
+
+public class HiveDateGranularity extends SqlFunction {
+
+ public static final SqlFunction YEAR = new HiveDateGranularity("YEAR");
+ public static final SqlFunction QUARTER = new HiveDateGranularity("QUARTER");
+ public static final SqlFunction MONTH = new HiveDateGranularity("MONTH");
+ public static final SqlFunction WEEK = new HiveDateGranularity("WEEK");
+ public static final SqlFunction DAY = new HiveDateGranularity("DAY");
+ public static final SqlFunction HOUR = new HiveDateGranularity("HOUR");
+ public static final SqlFunction MINUTE = new HiveDateGranularity("MINUTE");
+ public static final SqlFunction SECOND = new HiveDateGranularity("SECOND");
+
+ public static final Set<SqlFunction> ALL_FUNCTIONS =
+ Sets.newHashSet(YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE, SECOND);
+
+ private HiveDateGranularity(String name) {
+ super(
+ name,
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.TIME_NULLABLE,
+ null,
+ OperandTypes.ANY,
+ SqlFunctionCategory.TIMEDATE);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
index aac6126..fd19d99 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveProjectSortTransposeRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.RelCollation;
import org.apache.calcite.rel.RelCollationTraitDef;
@@ -48,6 +49,10 @@ public class HiveProjectSortTransposeRule extends RelOptRule {
operand(HiveSortLimit.class, any())));
}
+ protected HiveProjectSortTransposeRule(RelOptRuleOperand operand) {
+ super(operand);
+ }
+
//~ Methods ----------------------------------------------------------------
// implement RelOptRule
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
index feec3c2..fe29850 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/rules/HiveSortProjectTransposeRule.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hive.ql.optimizer.calcite.rules;
import org.apache.calcite.plan.RelOptRule;
import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
import org.apache.calcite.plan.RelOptUtil;
import org.apache.calcite.rel.RelCollation;
import org.apache.calcite.rel.RelCollationTraitDef;
@@ -49,6 +50,10 @@ public class HiveSortProjectTransposeRule extends RelOptRule {
operand(HiveProject.class, any())));
}
+ protected HiveSortProjectTransposeRule(RelOptRuleOperand operand) {
+ super(operand);
+ }
+
//~ Methods ----------------------------------------------------------------
@Override
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
index 78c76ab..9a5becb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTBuilder.java
@@ -22,19 +22,21 @@ import java.text.DateFormat;
import java.text.SimpleDateFormat;
import java.util.Calendar;
-import org.apache.calcite.avatica.util.ByteString;
import org.apache.calcite.rel.core.JoinRelType;
import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rex.RexLiteral;
import org.apache.calcite.sql.type.SqlTypeName;
import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
+import org.apache.hadoop.hive.conf.Constants;
import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveTableScan;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
import org.apache.hadoop.hive.ql.parse.HiveParser;
import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.SemanticAnalyzer;
class ASTBuilder {
@@ -62,14 +64,32 @@ class ASTBuilder {
ASTBuilder.construct(HiveParser.TOK_TABNAME, "TOK_TABNAME")
.add(HiveParser.Identifier, hTbl.getHiveTableMD().getDbName())
.add(HiveParser.Identifier, hTbl.getHiveTableMD().getTableName()));
- // we need to carry the insideView information from calcite into the ast.
- if (((HiveTableScan) scan).isInsideView()) {
- b.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTIES, "TOK_TABLEPROPERTIES").add(
- ASTBuilder.construct(HiveParser.TOK_TABLEPROPLIST, "TOK_TABLEPROPLIST").add(
- ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
- .add(HiveParser.StringLiteral, "\"insideView\"")
- .add(HiveParser.StringLiteral, "\"TRUE\""))));
+
+ HiveTableScan hts;
+ if (scan instanceof DruidQuery) {
+ hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
+ } else {
+ hts = (HiveTableScan) scan;
+ }
+ ASTBuilder propList = ASTBuilder.construct(HiveParser.TOK_TABLEPROPLIST, "TOK_TABLEPROPLIST");
+ if (scan instanceof DruidQuery) {
+ // Pass possible query to Druid
+ DruidQuery dq = (DruidQuery) scan;
+ propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+ .add(HiveParser.StringLiteral, "\"" + Constants.DRUID_QUERY_JSON + "\"")
+ .add(HiveParser.StringLiteral, "\"" + SemanticAnalyzer.escapeSQLString(
+ dq.getQueryString()) + "\""));
+ propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+ .add(HiveParser.StringLiteral, "\"" + Constants.DRUID_QUERY_TYPE + "\"")
+ .add(HiveParser.StringLiteral, "\"" + dq.getQueryType() + "\""));
+ }
+ if (hts.isInsideView()) {
+ // We need to carry the insideView information from calcite into the ast.
+ propList.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTY, "TOK_TABLEPROPERTY")
+ .add(HiveParser.StringLiteral, "\"insideView\"")
+ .add(HiveParser.StringLiteral, "\"TRUE\""));
}
+ b.add(ASTBuilder.construct(HiveParser.TOK_TABLEPROPERTIES, "TOK_TABLEPROPERTIES").add(propList));
// NOTE: Calcite considers tbls to be equal if their names are the same. Hence
// we need to provide Calcite the fully qualified table name (dbname.tblname)
@@ -77,7 +97,7 @@ class ASTBuilder {
// However in HIVE DB name can not appear in select list; in case of join
// where table names differ only in DB name, Hive would require user
// introducing explicit aliases for tbl.
- b.add(HiveParser.Identifier, ((HiveTableScan)scan).getTableAlias());
+ b.add(HiveParser.Identifier, hts.getTableAlias());
return b.node();
}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
index 40215a2..9f5e733 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/ASTConverter.java
@@ -56,6 +56,7 @@ import org.apache.calcite.util.ImmutableBitSet;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
import org.apache.hadoop.hive.ql.metadata.VirtualColumn;
import org.apache.hadoop.hive.ql.optimizer.calcite.CalciteSemanticException;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveSortLimit;
@@ -625,7 +626,13 @@ public class ASTConverter {
private static final long serialVersionUID = 1L;
Schema(TableScan scan) {
- String tabName = ((HiveTableScan) scan).getTableAlias();
+ HiveTableScan hts;
+ if (scan instanceof DruidQuery) {
+ hts = (HiveTableScan) ((DruidQuery)scan).getTableScan();
+ } else {
+ hts = (HiveTableScan) scan;
+ }
+ String tabName = hts.getTableAlias();
for (RelDataTypeField field : scan.getRowType().getFieldList()) {
add(new ColumnInfo(tabName, field.getName()));
}
[3/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Posted by jc...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
index 8b08ae7..911b86b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/SqlFunctionConverter.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlCountAggFunc
import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlMinMaxAggFunction;
import org.apache.hadoop.hive.ql.optimizer.calcite.functions.HiveSqlSumAggFunction;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveBetween;
+import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveDateGranularity;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveIn;
import org.apache.hadoop.hive.ql.parse.ASTNode;
import org.apache.hadoop.hive.ql.parse.HiveParser;
@@ -206,6 +207,7 @@ public class SqlFunctionConverter {
case IS_NOT_NULL:
case IS_NULL:
case CASE:
+ case OTHER_FUNCTION:
node = (ASTNode) ParseDriver.adaptor.create(HiveParser.TOK_FUNCTION, "TOK_FUNCTION");
node.addChild((ASTNode) ParseDriver.adaptor.create(hToken.type, hToken.text));
break;
@@ -311,8 +313,8 @@ public class SqlFunctionConverter {
registerFunction("+", SqlStdOperatorTable.PLUS, hToken(HiveParser.PLUS, "+"));
registerFunction("-", SqlStdOperatorTable.MINUS, hToken(HiveParser.MINUS, "-"));
registerFunction("*", SqlStdOperatorTable.MULTIPLY, hToken(HiveParser.STAR, "*"));
- registerFunction("/", SqlStdOperatorTable.DIVIDE, hToken(HiveParser.STAR, "/"));
- registerFunction("%", SqlStdOperatorTable.MOD, hToken(HiveParser.STAR, "%"));
+ registerFunction("/", SqlStdOperatorTable.DIVIDE, hToken(HiveParser.DIVIDE, "/"));
+ registerFunction("%", SqlStdOperatorTable.MOD, hToken(HiveParser.Identifier, "%"));
registerFunction("and", SqlStdOperatorTable.AND, hToken(HiveParser.KW_AND, "and"));
registerFunction("or", SqlStdOperatorTable.OR, hToken(HiveParser.KW_OR, "or"));
registerFunction("=", SqlStdOperatorTable.EQUALS, hToken(HiveParser.EQUAL, "="));
@@ -334,6 +336,23 @@ public class SqlFunctionConverter {
registerFunction("isnull", SqlStdOperatorTable.IS_NULL, hToken(HiveParser.TOK_ISNULL, "TOK_ISNULL"));
registerFunction("when", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
registerDuplicateFunction("case", SqlStdOperatorTable.CASE, hToken(HiveParser.Identifier, "when"));
+ // timebased
+ registerFunction("floor_year", HiveDateGranularity.YEAR,
+ hToken(HiveParser.Identifier, "floor_year"));
+ registerFunction("floor_quarter", HiveDateGranularity.QUARTER,
+ hToken(HiveParser.Identifier, "floor_quarter"));
+ registerFunction("floor_month", HiveDateGranularity.MONTH,
+ hToken(HiveParser.Identifier, "floor_month"));
+ registerFunction("floor_week", HiveDateGranularity.WEEK,
+ hToken(HiveParser.Identifier, "floor_week"));
+ registerFunction("floor_day", HiveDateGranularity.DAY,
+ hToken(HiveParser.Identifier, "floor_day"));
+ registerFunction("floor_hour", HiveDateGranularity.HOUR,
+ hToken(HiveParser.Identifier, "floor_hour"));
+ registerFunction("floor_minute", HiveDateGranularity.MINUTE,
+ hToken(HiveParser.Identifier, "floor_minute"));
+ registerFunction("floor_second", HiveDateGranularity.SECOND,
+ hToken(HiveParser.Identifier, "floor_second"));
}
private void registerFunction(String name, SqlOperator calciteFn, HiveToken hiveToken) {
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
index 18a55cb..ff94160 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/CalcitePlanner.java
@@ -24,6 +24,7 @@ import java.lang.reflect.UndeclaredThrowableException;
import java.math.BigDecimal;
import java.util.AbstractMap.SimpleEntry;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.BitSet;
import java.util.Collections;
import java.util.EnumSet;
@@ -61,6 +62,7 @@ import org.apache.calcite.rel.core.Aggregate;
import org.apache.calcite.rel.core.AggregateCall;
import org.apache.calcite.rel.core.Filter;
import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.core.TableScan;
import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
import org.apache.calcite.rel.metadata.RelMetadataProvider;
@@ -98,6 +100,7 @@ import org.apache.calcite.util.ImmutableBitSet;
import org.apache.calcite.util.ImmutableIntList;
import org.apache.calcite.util.Pair;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.Constants;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
import org.apache.hadoop.hive.conf.HiveConf.StrictChecks;
@@ -128,6 +131,11 @@ import org.apache.hadoop.hive.ql.optimizer.calcite.RelOptHiveTable;
import org.apache.hadoop.hive.ql.optimizer.calcite.TraitsUtil;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveAlgorithmsConf;
import org.apache.hadoop.hive.ql.optimizer.calcite.cost.HiveVolcanoPlanner;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidQuery;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidRules;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidSchema;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.HiveDruidConf;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveAggregate;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveFilter;
import org.apache.hadoop.hive.ql.optimizer.calcite.reloperators.HiveGroupingID;
@@ -208,6 +216,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.joda.time.Interval;
import com.google.common.base.Function;
import com.google.common.collect.ImmutableList;
@@ -931,8 +940,11 @@ public class CalcitePlanner extends SemanticAnalyzer {
final Double maxMemory = (double) HiveConf.getLongVar(
conf, HiveConf.ConfVars.HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD);
HiveAlgorithmsConf algorithmsConf = new HiveAlgorithmsConf(maxSplitSize, maxMemory);
+ final int selectThreshold = (int) HiveConf.getIntVar(
+ conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
+ HiveDruidConf druidConf = new HiveDruidConf(selectThreshold);
HiveRulesRegistry registry = new HiveRulesRegistry();
- HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, registry);
+ HivePlannerContext confContext = new HivePlannerContext(algorithmsConf, druidConf, registry);
RelOptPlanner planner = HiveVolcanoPlanner.createPlanner(confContext);
final RelOptQuery query = new RelOptQuery(planner);
final RexBuilder rexBuilder = cluster.getRexBuilder();
@@ -1070,10 +1082,18 @@ public class CalcitePlanner extends SemanticAnalyzer {
perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Window fixing rule");
}
- // 8. Run rules to aid in translation from Calcite tree to Hive tree
+ // 8. Apply Druid transformation rules
+ perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
+ calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
+ HepMatchOrder.BOTTOM_UP, DruidRules.FILTER, DruidRules.PROJECT_AGGREGATE,
+ DruidRules.PROJECT, DruidRules.AGGREGATE, DruidRules.PROJECT_SORT,
+ DruidRules.SORT, DruidRules.SORT_PROJECT);
+ perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER, "Calcite: Druid transformation rules");
+
+ // 9. Run rules to aid in translation from Calcite tree to Hive tree
if (HiveConf.getBoolVar(conf, ConfVars.HIVE_CBO_RETPATH_HIVEOP)) {
perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.OPTIMIZER);
- // 8.1. Merge join into multijoin operators (if possible)
+ // 9.1. Merge join into multijoin operators (if possible)
calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, true, mdProvider.getMetadataProvider(), null,
HepMatchOrder.BOTTOM_UP, HiveJoinProjectTransposeRule.BOTH_PROJECT_INCLUDE_OUTER,
HiveJoinProjectTransposeRule.LEFT_PROJECT_INCLUDE_OUTER,
@@ -1091,7 +1111,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
new HiveFilterProjectTSTransposeRule(Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY,
HiveProject.class, HiveRelFactories.HIVE_PROJECT_FACTORY, HiveTableScan.class));
- // 8.2. Introduce exchange operators below join/multijoin operators
+ // 9.2. Introduce exchange operators below join/multijoin operators
calciteOptimizedPlan = hepPlan(calciteOptimizedPlan, false, mdProvider.getMetadataProvider(), null,
HepMatchOrder.BOTTOM_UP, HiveInsertExchange4JoinRule.EXCHANGE_BELOW_JOIN,
HiveInsertExchange4JoinRule.EXCHANGE_BELOW_MULTIJOIN);
@@ -1251,7 +1271,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
basePlan = hepPlan(basePlan, true, mdProvider, null,
new HiveFilterProjectTSTransposeRule(
Filter.class, HiveRelFactories.HIVE_FILTER_FACTORY, HiveProject.class,
- HiveRelFactories.HIVE_PROJECT_FACTORY, HiveTableScan.class),
+ HiveRelFactories.HIVE_PROJECT_FACTORY, TableScan.class),
HiveProjectFilterPullUpConstantsRule.INSTANCE);
perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.OPTIMIZER,
"Calcite: Prejoin ordering transformation, Rerun PPD");
@@ -1657,7 +1677,7 @@ public class CalcitePlanner extends SemanticAnalyzer {
private RelNode genTableLogicalPlan(String tableAlias, QB qb) throws SemanticException {
RowResolver rr = new RowResolver();
- HiveTableScan tableRel = null;
+ RelNode tableRel = null;
try {
@@ -1713,16 +1733,20 @@ public class CalcitePlanner extends SemanticAnalyzer {
partitionColumns.add(colInfo);
}
+ final TableType tableType = obtainTableType(tabMetaData);
+
// 3.3 Add column info corresponding to virtual columns
List<VirtualColumn> virtualCols = new ArrayList<VirtualColumn>();
- Iterator<VirtualColumn> vcs = VirtualColumn.getRegistry(conf).iterator();
- while (vcs.hasNext()) {
- VirtualColumn vc = vcs.next();
- colInfo = new ColumnInfo(vc.getName(), vc.getTypeInfo(), tableAlias, true,
- vc.getIsHidden());
- rr.put(tableAlias, vc.getName().toLowerCase(), colInfo);
- cInfoLst.add(colInfo);
- virtualCols.add(vc);
+ if (tableType == TableType.NATIVE) {
+ Iterator<VirtualColumn> vcs = VirtualColumn.getRegistry(conf).iterator();
+ while (vcs.hasNext()) {
+ VirtualColumn vc = vcs.next();
+ colInfo = new ColumnInfo(vc.getName(), vc.getTypeInfo(), tableAlias, true,
+ vc.getIsHidden());
+ rr.put(tableAlias, vc.getName().toLowerCase(), colInfo);
+ cInfoLst.add(colInfo);
+ virtualCols.add(vc);
+ }
}
// 3.4 Build row type from field <type, name>
@@ -1737,15 +1761,50 @@ public class CalcitePlanner extends SemanticAnalyzer {
fullyQualifiedTabName = tabMetaData.getTableName();
}
RelOptHiveTable optTable = new RelOptHiveTable(relOptSchema, fullyQualifiedTabName,
- rowType, tabMetaData, nonPartitionColumns, partitionColumns, virtualCols, conf,
- partitionCache, noColsMissingStats);
-
- // 5. Build Hive Table Scan Rel
- tableRel = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), optTable,
- null == tableAlias ? tabMetaData.getTableName() : tableAlias,
- getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
- HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
- || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+ rowType, tabMetaData, nonPartitionColumns, partitionColumns, virtualCols, conf,
+ partitionCache, noColsMissingStats);
+
+ // 5. Build operator
+ if (tableType == TableType.DRUID) {
+ // Build Druid query
+ String address = HiveConf.getVar(conf,
+ HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+ String dataSource = tabMetaData.getParameters().get(Constants.DRUID_DATA_SOURCE);
+ Set<String> metrics = new HashSet<>();
+ List<RelDataType> druidColTypes = new ArrayList<>();
+ List<String> druidColNames = new ArrayList<>();
+ for (RelDataTypeField field : rowType.getFieldList()) {
+ druidColTypes.add(field.getType());
+ druidColNames.add(field.getName());
+ if (field.getName().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+ // timestamp
+ continue;
+ }
+ if (field.getType().getSqlTypeName() == SqlTypeName.VARCHAR) {
+ // dimension
+ continue;
+ }
+ metrics.add(field.getName());
+ }
+ List<Interval> intervals = Arrays.asList(DruidTable.DEFAULT_INTERVAL);
+
+ DruidTable druidTable = new DruidTable(new DruidSchema(address),
+ dataSource, rowType, metrics, intervals, DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+ final TableScan scan = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+ optTable, null == tableAlias ? tabMetaData.getTableName() : tableAlias,
+ getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
+ HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
+ || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+ tableRel = DruidQuery.create(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION),
+ optTable, druidTable, ImmutableList.<RelNode>of(scan));
+ } else {
+ // Build Hive Table Scan Rel
+ tableRel = new HiveTableScan(cluster, cluster.traitSetOf(HiveRelNode.CONVENTION), optTable,
+ null == tableAlias ? tabMetaData.getTableName() : tableAlias,
+ getAliasId(tableAlias, qb), HiveConf.getBoolVar(conf,
+ HiveConf.ConfVars.HIVE_CBO_RETPATH_HIVEOP), qb.isInsideView()
+ || qb.getAliasInsideView().contains(tableAlias.toLowerCase()));
+ }
// 6. Add Schema(RR) to RelNode-Schema map
ImmutableMap<String, Integer> hiveToCalciteColMap = buildHiveToCalciteColumnMap(rr,
@@ -1763,6 +1822,15 @@ public class CalcitePlanner extends SemanticAnalyzer {
return tableRel;
}
+ private TableType obtainTableType(Table tabMetaData) {
+ if (tabMetaData.getStorageHandler() != null &&
+ tabMetaData.getStorageHandler().toString().equals(
+ Constants.DRUID_HIVE_STORAGE_HANDLER_ID)) {
+ return TableType.DRUID;
+ }
+ return TableType.NATIVE;
+ }
+
private RelNode genFilterRelNode(ASTNode filterExpr, RelNode srcRel,
boolean useCaching) throws SemanticException {
ExprNodeDesc filterCondn = genExprNodeDesc(filterExpr, relToHiveRR.get(srcRel), useCaching);
@@ -3366,4 +3434,9 @@ public class CalcitePlanner extends SemanticAnalyzer {
}
}
+ private enum TableType {
+ DRUID,
+ NATIVE
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index ace0e9c..943d9d7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -56,7 +56,6 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.hive.common.BlobStorageUtils;
-import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.ObjectPair;
import org.apache.hadoop.hive.common.StatsSetupConst;
@@ -156,6 +155,7 @@ import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFrameSpec;
import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowFunctionSpec;
import org.apache.hadoop.hive.ql.parse.WindowingSpec.WindowSpec;
import org.apache.hadoop.hive.ql.plan.AggregationDesc;
+import org.apache.hadoop.hive.ql.plan.AlterTableDesc;
import org.apache.hadoop.hive.ql.plan.AlterTableDesc.AlterTableTypes;
import org.apache.hadoop.hive.ql.plan.CreateTableDesc;
import org.apache.hadoop.hive.ql.plan.CreateTableLikeDesc;
@@ -9787,11 +9787,24 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
// is the table already present
TableScanOperator top = topOps.get(alias_id);
+ // Obtain table props in query
+ Map<String, String> properties = qb.getTabPropsForAlias(alias);
+
if (top == null) {
// Determine row schema for TSOP.
// Include column names from SerDe, the partition and virtual columns.
rwsch = new RowResolver();
try {
+ // Including parameters passed in the query
+ if (properties != null) {
+ for (Entry<String, String> prop : properties.entrySet()) {
+ if (tab.getSerdeParam(prop.getKey()) != null) {
+ LOG.warn("SerDe property in input query overrides stored SerDe property");
+ }
+ tab.setSerdeParam(prop.getKey(), prop.getValue());
+ }
+ }
+ // Obtain inspector for schema
StructObjectInspector rowObjectInspector = (StructObjectInspector) tab
.getDeserializer().getObjectInspector();
List<? extends StructField> fields = rowObjectInspector
@@ -9852,10 +9865,9 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
// Add a mapping from the table scan operator to Table
topToTable.put(top, tab);
- Map<String, String> props = qb.getTabPropsForAlias(alias);
- if (props != null) {
- topToTableProps.put(top, props);
- tsDesc.setOpProps(props);
+ if (properties != null) {
+ topToTableProps.put(top, properties);
+ tsDesc.setOpProps(properties);
}
} else {
rwsch = opParseCtx.get(top).getRowResolver();
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
index bf808c3..eafba21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateTableDesc.java
@@ -24,14 +24,15 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.TableType;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
-import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
import org.apache.hadoop.hive.metastore.api.Order;
+import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
+import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
import org.apache.hadoop.hive.ql.ErrorMsg;
import org.apache.hadoop.hive.ql.exec.DDLTask;
import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -476,7 +477,8 @@ public class CreateTableDesc extends DDLDesc implements Serializable {
if ((this.getCols() == null) || (this.getCols().size() == 0)) {
// for now make sure that serde exists
- if (Table.hasMetastoreBasedSchema(conf, getSerName())) {
+ if (Table.hasMetastoreBasedSchema(conf, serName) &&
+ StringUtils.isEmpty(getStorageHandler())) {
throw new SemanticException(ErrorMsg.INVALID_TBL_DDL_SERDE.getMsg());
}
return;
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
index 47c65bb..ebe613e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
@@ -389,4 +389,11 @@ public class TableScanDesc extends AbstractOperatorDesc {
}
return rtn;
}
+
+ @Override
+ @Explain(displayName = "properties", explainLevels = { Level.DEFAULT, Level.USER, Level.EXTENDED })
+ public Map<String, String> getOpProps() {
+ return opProps;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
new file mode 100644
index 0000000..08ed9fd
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloor.java
@@ -0,0 +1,506 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hive.ql.exec.UDF;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.joda.time.Chronology;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
+import org.joda.time.Period;
+import org.joda.time.ReadableDuration;
+import org.joda.time.chrono.ISOChronology;
+
+import com.google.common.collect.ImmutableMap;
+
+/**
+ * UDFDateFloor.
+ *
+ * Abstract class that converts a timestamp to a timestamp with a given granularity.
+ */
+public abstract class UDFDateFloor extends UDF {
+
+ private final QueryGranularity granularity;
+
+ private final TimestampWritable result;
+
+ public UDFDateFloor(String granularity) {
+ this.granularity = QueryGranularity.fromString(granularity);
+ this.result = new TimestampWritable();
+ }
+
+ public TimestampWritable evaluate(TimestampWritable t) {
+ if (t == null) {
+ return null;
+ }
+ long newTimestamp = granularity.truncate(t.getTimestamp().getTime());
+ result.setTime(newTimestamp);
+ return result;
+ }
+
+ /*
+ * This code that creates the result for the granularity functions has been brought from Druid
+ */
+
+ private static final Map<String, PeriodGranularity> CALENDRIC_GRANULARITIES = ImmutableMap.of(
+ "YEAR", new PeriodGranularity(new Period("P1Y"), null, null),
+ "MONTH", new PeriodGranularity(new Period("P1M"), null, null),
+ "QUARTER", new PeriodGranularity(new Period("P3M"), null, null),
+ "WEEK", new PeriodGranularity(new Period("P1W"), null, null));
+
+ private static abstract class QueryGranularity {
+
+ public abstract long next(long offset);
+
+ public abstract long truncate(long offset);
+
+ public abstract DateTime toDateTime(long offset);
+
+ public abstract Iterable<Long> iterable(final long start, final long end);
+
+ public static QueryGranularity fromString(String str) {
+ String name = str.toUpperCase();
+ if (CALENDRIC_GRANULARITIES.containsKey(name)) {
+ return CALENDRIC_GRANULARITIES.get(name);
+ }
+ return new DurationGranularity(convertValue(str), 0);
+ }
+
+ private static enum MillisIn {
+ SECOND(1000), MINUTE(60 * 1000), FIFTEEN_MINUTE(15 * 60 * 1000), THIRTY_MINUTE(
+ 30 * 60 * 1000), HOUR(3600 * 1000), DAY(24 * 3600 * 1000);
+
+ private final long millis;
+
+ MillisIn(final long millis) {
+ this.millis = millis;
+ }
+ }
+
+ private static long convertValue(Object o) {
+ if (o instanceof String) {
+ return MillisIn.valueOf(((String) o).toUpperCase()).millis;
+ } else if (o instanceof ReadableDuration) {
+ return ((ReadableDuration) o).getMillis();
+ } else if (o instanceof Number) {
+ return ((Number) o).longValue();
+ }
+ throw new RuntimeException("Granularity not recognized");
+ }
+ }
+
+ private static abstract class BaseQueryGranularity extends QueryGranularity {
+ public abstract long next(long offset);
+
+ public abstract long truncate(long offset);
+
+ public DateTime toDateTime(long offset) {
+ return new DateTime(offset, DateTimeZone.UTC);
+ }
+
+ public Iterable<Long> iterable(final long start, final long end) {
+ return new Iterable<Long>() {
+ @Override
+ public Iterator<Long> iterator() {
+ return new Iterator<Long>() {
+ long curr = truncate(start);
+
+ long next = BaseQueryGranularity.this.next(curr);
+
+ @Override
+ public boolean hasNext() {
+ return curr < end;
+ }
+
+ @Override
+ public Long next() {
+ if (!hasNext()) {
+ throw new NoSuchElementException();
+ }
+
+ long retVal = curr;
+
+ curr = next;
+ next = BaseQueryGranularity.this.next(curr);
+
+ return retVal;
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException();
+ }
+ };
+ }
+ };
+ }
+ }
+
+ private static class PeriodGranularity extends BaseQueryGranularity {
+ private final Period period;
+
+ private final Chronology chronology;
+
+ private final long origin;
+
+ private final boolean hasOrigin;
+
+ private final boolean isCompound;
+
+ public PeriodGranularity(Period period, DateTime origin, DateTimeZone tz) {
+ this.period = period;
+ this.chronology = tz == null ? ISOChronology.getInstanceUTC() : ISOChronology.getInstance(tz);
+ if (origin == null) {
+ // default to origin in given time zone when aligning multi-period granularities
+ this.origin = new DateTime(0, DateTimeZone.UTC).withZoneRetainFields(chronology.getZone())
+ .getMillis();
+ this.hasOrigin = false;
+ } else {
+ this.origin = origin.getMillis();
+ this.hasOrigin = true;
+ }
+ this.isCompound = isCompoundPeriod(period);
+ }
+
+ @Override
+ public DateTime toDateTime(long t) {
+ return new DateTime(t, chronology.getZone());
+ }
+
+ @Override
+ public long next(long t) {
+ return chronology.add(period, t, 1);
+ }
+
+ @Override
+ public long truncate(long t) {
+ if (isCompound) {
+ try {
+ return truncateMillisPeriod(t);
+ } catch (UnsupportedOperationException e) {
+ return truncateCompoundPeriod(t);
+ }
+ }
+
+ final int years = period.getYears();
+ if (years > 0) {
+ if (years > 1 || hasOrigin) {
+ int y = chronology.years().getDifference(t, origin);
+ y -= y % years;
+ long tt = chronology.years().add(origin, y);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.years().add(tt, -years);
+ else
+ t = tt;
+ return t;
+ } else {
+ return chronology.year().roundFloor(t);
+ }
+ }
+
+ final int months = period.getMonths();
+ if (months > 0) {
+ if (months > 1 || hasOrigin) {
+ int m = chronology.months().getDifference(t, origin);
+ m -= m % months;
+ long tt = chronology.months().add(origin, m);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.months().add(tt, -months);
+ else
+ t = tt;
+ return t;
+ } else {
+ return chronology.monthOfYear().roundFloor(t);
+ }
+ }
+
+ final int weeks = period.getWeeks();
+ if (weeks > 0) {
+ if (weeks > 1 || hasOrigin) {
+ // align on multiples from origin
+ int w = chronology.weeks().getDifference(t, origin);
+ w -= w % weeks;
+ long tt = chronology.weeks().add(origin, w);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.weeks().add(tt, -weeks);
+ else
+ t = tt;
+ return t;
+ } else {
+ t = chronology.dayOfWeek().roundFloor(t);
+ // default to Monday as beginning of the week
+ return chronology.dayOfWeek().set(t, 1);
+ }
+ }
+
+ final int days = period.getDays();
+ if (days > 0) {
+ if (days > 1 || hasOrigin) {
+ // align on multiples from origin
+ int d = chronology.days().getDifference(t, origin);
+ d -= d % days;
+ long tt = chronology.days().add(origin, d);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.days().add(tt, -days);
+ else
+ t = tt;
+ return t;
+ } else {
+ t = chronology.hourOfDay().roundFloor(t);
+ return chronology.hourOfDay().set(t, 0);
+ }
+ }
+
+ final int hours = period.getHours();
+ if (hours > 0) {
+ if (hours > 1 || hasOrigin) {
+ // align on multiples from origin
+ long h = chronology.hours().getDifferenceAsLong(t, origin);
+ h -= h % hours;
+ long tt = chronology.hours().add(origin, h);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.hours().add(tt, -hours);
+ else
+ t = tt;
+ return t;
+ } else {
+ t = chronology.minuteOfHour().roundFloor(t);
+ return chronology.minuteOfHour().set(t, 0);
+ }
+ }
+
+ final int minutes = period.getMinutes();
+ if (minutes > 0) {
+ // align on multiples from origin
+ if (minutes > 1 || hasOrigin) {
+ long m = chronology.minutes().getDifferenceAsLong(t, origin);
+ m -= m % minutes;
+ long tt = chronology.minutes().add(origin, m);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.minutes().add(tt, -minutes);
+ else
+ t = tt;
+ return t;
+ } else {
+ t = chronology.secondOfMinute().roundFloor(t);
+ return chronology.secondOfMinute().set(t, 0);
+ }
+ }
+
+ final int seconds = period.getSeconds();
+ if (seconds > 0) {
+ // align on multiples from origin
+ if (seconds > 1 || hasOrigin) {
+ long s = chronology.seconds().getDifferenceAsLong(t, origin);
+ s -= s % seconds;
+ long tt = chronology.seconds().add(origin, s);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.seconds().add(tt, -seconds);
+ else
+ t = tt;
+ return t;
+ } else {
+ return chronology.millisOfSecond().set(t, 0);
+ }
+ }
+
+ final int millis = period.getMillis();
+ if (millis > 0) {
+ if (millis > 1) {
+ long ms = chronology.millis().getDifferenceAsLong(t, origin);
+ ms -= ms % millis;
+ long tt = chronology.millis().add(origin, ms);
+ // always round down to the previous period (for timestamps prior to origin)
+ if (t < tt)
+ t = chronology.millis().add(tt, -millis);
+ else
+ t = tt;
+ return t;
+ } else {
+ return t;
+ }
+ }
+
+ return t;
+ }
+
+ private static boolean isCompoundPeriod(Period period) {
+ int[] values = period.getValues();
+ boolean single = false;
+ for (int v : values) {
+ if (v > 0) {
+ if (single)
+ return true;
+ single = true;
+ }
+ }
+ return false;
+ }
+
+ private long truncateCompoundPeriod(long t) {
+ long current;
+ if (t >= origin) {
+ long next = origin;
+ do {
+ current = next;
+ next = chronology.add(period, current, 1);
+ } while (t >= next);
+ } else {
+ current = origin;
+ do {
+ current = chronology.add(period, current, -1);
+ } while (t < current);
+ }
+ return current;
+ }
+
+ private long truncateMillisPeriod(final long t) {
+ // toStandardDuration assumes days are always 24h, and hours are always 60 minutes,
+ // which may not always be the case, e.g if there are daylight saving changes.
+ if (chronology.days().isPrecise() && chronology.hours().isPrecise()) {
+ final long millis = period.toStandardDuration().getMillis();
+ long offset = t % millis - origin % millis;
+ if (offset < 0) {
+ offset += millis;
+ }
+ return t - offset;
+ } else {
+ throw new UnsupportedOperationException(
+ "Period cannot be converted to milliseconds as some fields mays vary in length with chronology "
+ + chronology.toString());
+ }
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ PeriodGranularity that = (PeriodGranularity) o;
+
+ if (hasOrigin != that.hasOrigin) {
+ return false;
+ }
+ if (origin != that.origin) {
+ return false;
+ }
+ if (!chronology.equals(that.chronology)) {
+ return false;
+ }
+ if (!period.equals(that.period)) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = period.hashCode();
+ result = 31 * result + chronology.hashCode();
+ result = 31 * result + (int) (origin ^ (origin >>> 32));
+ result = 31 * result + (hasOrigin ? 1 : 0);
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "PeriodGranularity{" + "period=" + period + ", timeZone=" + chronology.getZone()
+ + ", origin=" + (hasOrigin ? origin : "null") + '}';
+ }
+ }
+
+ private static class DurationGranularity extends BaseQueryGranularity {
+ private final long length;
+
+ private final long origin;
+
+ public DurationGranularity(long millis, long origin) {
+ this.length = millis;
+ this.origin = origin % length;
+ }
+
+ @Override
+ public long next(long t) {
+ return t + getDurationMillis();
+ }
+
+ @Override
+ public long truncate(final long t) {
+ final long duration = getDurationMillis();
+ long offset = t % duration - origin % duration;
+ if (offset < 0) {
+ offset += duration;
+ }
+ return t - offset;
+ }
+
+ public long getDurationMillis() {
+ return length;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ DurationGranularity that = (DurationGranularity) o;
+
+ if (length != that.length) {
+ return false;
+ }
+ if (origin != that.origin) {
+ return false;
+ }
+
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = (int) (length ^ (length >>> 32));
+ result = 31 * result + (int) (origin ^ (origin >>> 32));
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "DurationGranularity{" + "length=" + length + ", origin=" + origin + '}';
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
new file mode 100644
index 0000000..e205797
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorDay.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorDay.
+ *
+ * Converts a timestamp to a timestamp with day granularity.
+ */
+@Description(name = "floor_day",
+ value = "_FUNC_(param) - Returns the timestamp at a day granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-dd 00:00:00")
+public class UDFDateFloorDay extends UDFDateFloor {
+
+ public UDFDateFloorDay() {
+ super("DAY");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
new file mode 100644
index 0000000..84fd394
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorHour.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorHour.
+ *
+ * Converts a timestamp to a timestamp with hour granularity.
+ */
+@Description(name = "floor_hour",
+ value = "_FUNC_(param) - Returns the timestamp at a hour granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-dd HH:00:00")
+public class UDFDateFloorHour extends UDFDateFloor {
+
+ public UDFDateFloorHour() {
+ super("HOUR");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
new file mode 100644
index 0000000..45b8f7e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMinute.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorMinute.
+ *
+ * Converts a timestamp to a timestamp with minute granularity.
+ */
+@Description(name = "floor_minute",
+ value = "_FUNC_(param) - Returns the timestamp at a minute granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-dd HH:mm:00")
+public class UDFDateFloorMinute extends UDFDateFloor {
+
+ public UDFDateFloorMinute() {
+ super("MINUTE");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
new file mode 100644
index 0000000..e3c70a1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorMonth.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorMonth.
+ *
+ * Converts a timestamp to a timestamp with month granularity.
+ */
+@Description(name = "floor_month",
+ value = "_FUNC_(param) - Returns the timestamp at a month granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-01 00:00:00")
+public class UDFDateFloorMonth extends UDFDateFloor {
+
+ public UDFDateFloorMonth() {
+ super("MONTH");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
new file mode 100644
index 0000000..c017238
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorQuarter.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorQuarter.
+ *
+ * Converts a timestamp to a timestamp with quarter granularity.
+ */
+@Description(name = "floor_quarter",
+ value = "_FUNC_(param) - Returns the timestamp at a quarter granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-xx-01 00:00:00")
+public class UDFDateFloorQuarter extends UDFDateFloor {
+
+ public UDFDateFloorQuarter() {
+ super("QUARTER");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
new file mode 100644
index 0000000..5e10026
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorSecond.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorSecond.
+ *
+ * Converts a timestamp to a timestamp with second granularity.
+ */
+@Description(name = "floor_second",
+ value = "_FUNC_(param) - Returns the timestamp at a second granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-dd HH:mm:ss")
+public class UDFDateFloorSecond extends UDFDateFloor {
+
+ public UDFDateFloorSecond() {
+ super("SECOND");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
new file mode 100644
index 0000000..185a84a
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorWeek.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorWeek.
+ *
+ * Converts a timestamp to a timestamp with week granularity.
+ */
+@Description(name = "floor_week",
+ value = "_FUNC_(param) - Returns the timestamp at a week granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-MM-xx 00:00:00")
+public class UDFDateFloorWeek extends UDFDateFloor {
+
+ public UDFDateFloorWeek() {
+ super("WEEK");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
new file mode 100644
index 0000000..e68decf
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFDateFloorYear.java
@@ -0,0 +1,39 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+
+/**
+ * UDFDateFloorYear.
+ *
+ * Converts a timestamp to a timestamp with year granularity.
+ */
+@Description(name = "floor_year",
+ value = "_FUNC_(param) - Returns the timestamp at a year granularity",
+ extended = "param needs to be a timestamp value\n"
+ + "Example:\n "
+ + " > SELECT _FUNC_(CAST('yyyy-MM-dd HH:mm:ss' AS TIMESTAMP)) FROM src;\n"
+ + " yyyy-01-01 00:00:00")
+public class UDFDateFloorYear extends UDFDateFloor {
+
+ public UDFDateFloorYear() {
+ super("YEAR");
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
index 44e157b..2830f1f 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/optimizer/calcite/TestCBORuleFiredOnlyOnce.java
@@ -61,7 +61,7 @@ public class TestCBORuleFiredOnlyOnce {
// Create rules registry to not trigger a rule more than once
HiveRulesRegistry registry = new HiveRulesRegistry();
- HivePlannerContext context = new HivePlannerContext(null, registry);
+ HivePlannerContext context = new HivePlannerContext(null, null, registry);
HepPlanner planner = new HepPlanner(programBuilder.build(), context);
// Cluster
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
new file mode 100644
index 0000000..f871de2
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/udf/TestUDFDateFormatGranularity.java
@@ -0,0 +1,85 @@
+/**
+ * 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.hadoop.hive.ql.udf;
+
+import java.sql.Timestamp;
+
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.junit.Test;
+
+import junit.framework.TestCase;
+
+public class TestUDFDateFormatGranularity extends TestCase {
+
+ @Test
+ public void testTimestampToTimestampWithGranularity() throws Exception {
+ // Running example
+ // Friday 30th August 1985 02:47:02 AM
+ final TimestampWritable t = new TimestampWritable(new Timestamp(494218022082L));
+ UDFDateFloor g;
+
+ // Year granularity
+ // Tuesday 1st January 1985 12:00:00 AM
+ g = new UDFDateFloorYear();
+ TimestampWritable i1 = g.evaluate(t);
+ assertEquals(473385600000L, i1.getTimestamp().getTime());
+
+ // Quarter granularity
+ // Monday 1st July 1985 12:00:00 AM
+ g = new UDFDateFloorQuarter();
+ TimestampWritable i2 = g.evaluate(t);
+ assertEquals(489024000000L, i2.getTimestamp().getTime());
+
+ // Month granularity
+ // Thursday 1st August 1985 12:00:00 AM
+ g = new UDFDateFloorMonth();
+ TimestampWritable i3 = g.evaluate(t);
+ assertEquals(491702400000L, i3.getTimestamp().getTime());
+
+ // Week granularity
+ // Monday 26th August 1985 12:00:00 AM
+ g = new UDFDateFloorWeek();
+ TimestampWritable i4 = g.evaluate(t);
+ assertEquals(493862400000L, i4.getTimestamp().getTime());
+
+ // Day granularity
+ // Friday 30th August 1985 12:00:00 AM
+ g = new UDFDateFloorDay();
+ TimestampWritable i5 = g.evaluate(t);
+ assertEquals(494208000000L, i5.getTimestamp().getTime());
+
+ // Hour granularity
+ // Friday 30th August 1985 02:00:00 AM
+ g = new UDFDateFloorHour();
+ TimestampWritable i6 = g.evaluate(t);
+ assertEquals(494215200000L, i6.getTimestamp().getTime());
+
+ // Minute granularity
+ // Friday 30th August 1985 02:47:00 AM
+ g = new UDFDateFloorMinute();
+ TimestampWritable i7 = g.evaluate(t);
+ assertEquals(494218020000L, i7.getTimestamp().getTime());
+
+ // Second granularity
+ // Friday 30th August 1985 02:47:02 AM
+ g = new UDFDateFloorSecond();
+ TimestampWritable i8 = g.evaluate(t);
+ assertEquals(494218022000L, i8.getTimestamp().getTime());
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_address.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_address.q b/ql/src/test/queries/clientnegative/druid_address.q
new file mode 100644
index 0000000..35ba06a
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_address.q
@@ -0,0 +1,5 @@
+set hive.druid.broker.address.default=;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_buckets.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_buckets.q b/ql/src/test/queries/clientnegative/druid_buckets.q
new file mode 100644
index 0000000..6fc75d1
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_buckets.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+CLUSTERED BY (robot) INTO 32 BUCKETS
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_datasource.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_datasource.q b/ql/src/test/queries/clientnegative/druid_datasource.q
new file mode 100644
index 0000000..87481fd
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_datasource.q
@@ -0,0 +1,3 @@
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("property" = "localhost");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_external.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_external.q b/ql/src/test/queries/clientnegative/druid_external.q
new file mode 100644
index 0000000..2de04db
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_external.q
@@ -0,0 +1,5 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_location.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_location.q b/ql/src/test/queries/clientnegative/druid_location.q
new file mode 100644
index 0000000..a9705dc
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_location.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+LOCATION '/testfolder/'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientnegative/druid_partitions.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientnegative/druid_partitions.q b/ql/src/test/queries/clientnegative/druid_partitions.q
new file mode 100644
index 0000000..e26a3b6
--- /dev/null
+++ b/ql/src/test/queries/clientnegative/druid_partitions.q
@@ -0,0 +1,6 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+PARTITIONED BY (dt string)
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_basic1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_basic1.q b/ql/src/test/queries/clientpositive/druid_basic1.q
new file mode 100644
index 0000000..83f5968
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_basic1.q
@@ -0,0 +1,18 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_2;
+
+DROP TABLE druid_table_2;
+
+DROP TABLE druid_table_1;
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_basic2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_basic2.q b/ql/src/test/queries/clientpositive/druid_basic2.q
new file mode 100644
index 0000000..fe24410
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_basic2.q
@@ -0,0 +1,52 @@
+set hive.strict.checks.cartesian.product=false;
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- dimension
+EXPLAIN EXTENDED
+SELECT robot FROM druid_table_1;
+
+-- metric
+EXPLAIN EXTENDED
+SELECT delta FROM druid_table_1;
+
+EXPLAIN EXTENDED
+SELECT robot
+FROM druid_table_1
+WHERE language = 'en';
+
+EXPLAIN EXTENDED
+SELECT DISTINCT robot
+FROM druid_table_1
+WHERE language = 'en';
+
+-- TODO: currently nothing is pushed - ISNOTNULL
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1) a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+);
+
+EXPLAIN EXTENDED
+SELECT a.robot, b.language
+FROM
+(
+ (SELECT robot, language
+ FROM druid_table_1
+ WHERE language = 'en') a
+ JOIN
+ (SELECT language
+ FROM druid_table_1) b
+ ON a.language = b.language
+);
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_intervals.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_intervals.q b/ql/src/test/queries/clientpositive/druid_intervals.q
new file mode 100644
index 0000000..140ff82
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_intervals.q
@@ -0,0 +1,67 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- (-\u221e\u2025+\u221e)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1;
+
+-- (-\u221e\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` < '2012-03-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252012-03-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` >= '2010-01-01 00:00:00' AND `__time` <= '2012-03-01 00:00:00'
+ AND `__time` < '2011-01-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00';
+
+-- [2010-01-01 00:00:00\u20252011-01-01 00:00:00],[2012-01-01 00:00:00\u20252013-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2012-01-01 00:00:00' AND '2013-01-01 00:00:00');
+
+-- OVERLAP [2010-01-01 00:00:00\u20252012-01-01 00:00:00]
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE (`__time` BETWEEN '2010-01-01 00:00:00' AND '2011-01-01 00:00:00')
+ OR (`__time` BETWEEN '2010-06-01 00:00:00' AND '2012-01-01 00:00:00');
+
+-- IN: MULTIPLE INTERVALS [2010-01-01 00:00:00\u20252010-01-01 00:00:00),[2011-01-01 00:00:00\u20252011-01-01 00:00:00)
+EXPLAIN
+SELECT `__time`
+FROM druid_table_1
+WHERE `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');
+
+EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' AND `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');
+
+EXPLAIN
+SELECT `__time`, robot
+FROM druid_table_1
+WHERE robot = 'user1' OR `__time` IN ('2010-01-01 00:00:00','2011-01-01 00:00:00');
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_timeseries.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_timeseries.q b/ql/src/test/queries/clientpositive/druid_timeseries.q
new file mode 100644
index 0000000..c0ad60f
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_timeseries.q
@@ -0,0 +1,94 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- GRANULARITY: ALL
+EXPLAIN
+SELECT max(added), sum(variation)
+FROM druid_table_1;
+
+-- GRANULARITY: NONE
+EXPLAIN
+SELECT `__time`, max(added), sum(variation)
+FROM druid_table_1
+GROUP BY `__time`;
+
+-- GRANULARITY: YEAR
+EXPLAIN
+SELECT floor_year(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_year(`__time`);
+
+-- GRANULARITY: QUARTER
+EXPLAIN
+SELECT floor_quarter(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_quarter(`__time`);
+
+-- GRANULARITY: MONTH
+EXPLAIN
+SELECT floor_month(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_month(`__time`);
+
+-- GRANULARITY: WEEK
+EXPLAIN
+SELECT floor_week(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_week(`__time`);
+
+-- GRANULARITY: DAY
+EXPLAIN
+SELECT floor_day(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_day(`__time`);
+
+-- GRANULARITY: HOUR
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_hour(`__time`);
+
+-- GRANULARITY: MINUTE
+EXPLAIN
+SELECT floor_minute(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_minute(`__time`);
+
+-- GRANULARITY: SECOND
+EXPLAIN
+SELECT floor_second(`__time`), max(added), sum(variation)
+FROM druid_table_1
+GROUP BY floor_second(`__time`);
+
+-- WITH FILTER ON DIMENSION
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY floor_hour(`__time`);
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT floor_hour(`__time`), max(added), sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY floor_hour(`__time`);
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT subq.h, subq.m, subq.s
+FROM
+(
+ SELECT floor_hour(`__time`) as h, max(added) as m, sum(variation) as s
+ FROM druid_table_1
+ GROUP BY floor_hour(`__time`)
+) subq
+WHERE subq.h BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP);
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/queries/clientpositive/druid_topn.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/druid_topn.q b/ql/src/test/queries/clientpositive/druid_topn.q
new file mode 100644
index 0000000..b121b7e
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/druid_topn.q
@@ -0,0 +1,75 @@
+set hive.druid.broker.address.default=localhost.test;
+
+CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia");
+
+DESCRIBE FORMATTED druid_table_1;
+
+-- GRANULARITY: ALL
+EXPLAIN
+SELECT robot, max(added) as m, sum(variation)
+FROM druid_table_1
+GROUP BY robot
+ORDER BY m DESC
+LIMIT 100;
+
+-- GRANULARITY: NONE
+EXPLAIN
+SELECT robot, `__time`, max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, `__time`
+ORDER BY s DESC
+LIMIT 100;
+
+-- GRANULARITY: YEAR
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s DESC
+LIMIT 10;
+
+-- ASC: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, floor_month(`__time`)
+ORDER BY s
+LIMIT 10;
+
+-- MULTIPLE ORDER: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY s DESC, m DESC
+LIMIT 10;
+
+-- MULTIPLE ORDER MIXED: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_month(`__time`), max(added) as m, sum(variation) as s
+FROM druid_table_1
+GROUP BY robot, namespace, floor_month(`__time`)
+ORDER BY robot ASC, m DESC
+LIMIT 10;
+
+-- WITH FILTER ON DIMENSION: TRANSFORM INTO GROUP BY
+EXPLAIN
+SELECT robot, floor_year(`__time`), max(added), sum(variation) as s
+FROM druid_table_1
+WHERE robot='1'
+GROUP BY robot, floor_year(`__time`)
+ORDER BY s
+LIMIT 10;
+
+-- WITH FILTER ON TIME
+EXPLAIN
+SELECT robot, floor_hour(`__time`), max(added) as m, sum(variation)
+FROM druid_table_1
+WHERE floor_hour(`__time`)
+ BETWEEN CAST('2010-01-01 00:00:00' AS TIMESTAMP)
+ AND CAST('2014-01-01 00:00:00' AS TIMESTAMP)
+GROUP BY robot, floor_hour(`__time`)
+ORDER BY m
+LIMIT 100;
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_address.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_address.q.out b/ql/src/test/results/clientnegative/druid_address.q.out
new file mode 100644
index 0000000..66b7e14
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_address.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Druid broker address not specified in configuration)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_buckets.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_buckets.q.out b/ql/src/test/results/clientnegative/druid_buckets.q.out
new file mode 100644
index 0000000..94e4f70
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_buckets.q.out
@@ -0,0 +1,8 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+CLUSTERED BY (robot) INTO 32 BUCKETS
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:CLUSTERED BY may not be specified for Druid)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_datasource.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_datasource.q.out b/ql/src/test/results/clientnegative/druid_datasource.q.out
new file mode 100644
index 0000000..177ffaa
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_datasource.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("property" = "localhost")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. java.lang.RuntimeException: MetaException(message:org.apache.hadoop.hive.serde2.SerDeException Druid data source not specified; use druid.datasource in table properties)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_external.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_external.q.out b/ql/src/test/results/clientnegative/druid_external.q.out
new file mode 100644
index 0000000..e5fac51
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_external.q.out
@@ -0,0 +1,7 @@
+PREHOOK: query: CREATE TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:Table in Druid needs to be declared as EXTERNAL)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_location.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_location.q.out b/ql/src/test/results/clientnegative/druid_location.q.out
new file mode 100644
index 0000000..5727e8c
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_location.q.out
@@ -0,0 +1,9 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+#### A masked pattern was here ####
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+#### A masked pattern was here ####
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:LOCATION may not be specified for Druid)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientnegative/druid_partitions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/druid_partitions.q.out b/ql/src/test/results/clientnegative/druid_partitions.q.out
new file mode 100644
index 0000000..6fb55c1
--- /dev/null
+++ b/ql/src/test/results/clientnegative/druid_partitions.q.out
@@ -0,0 +1,8 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+PARTITIONED BY (dt string)
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. MetaException(message:PARTITIONED BY may not be specified for Druid)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/create_view.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/create_view.q.out b/ql/src/test/results/clientpositive/create_view.q.out
index d9c1e11..7281185 100644
--- a/ql/src/test/results/clientpositive/create_view.q.out
+++ b/ql/src/test/results/clientpositive/create_view.q.out
@@ -190,6 +190,8 @@ STAGE PLANS:
Map Operator Tree:
TableScan
alias: src
+ properties:
+ insideView TRUE
Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
Filter Operator
predicate: (UDFToDouble(key) = 18.0) (type: boolean)
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/ql/src/test/results/clientpositive/druid_basic1.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/druid_basic1.q.out b/ql/src/test/results/clientpositive/druid_basic1.q.out
new file mode 100644
index 0000000..74ae9ed
--- /dev/null
+++ b/ql/src/test/results/clientpositive/druid_basic1.q.out
@@ -0,0 +1,142 @@
+PREHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: CREATE EXTERNAL TABLE druid_table_1
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_1
+PREHOOK: query: DESCRIBE FORMATTED druid_table_1
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_1
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_1
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_1
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: -- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@druid_table_2
+POSTHOOK: query: -- different table, same datasource
+CREATE EXTERNAL TABLE druid_table_2
+STORED BY 'org.apache.hadoop.hive.druid.QTestDruidStorageHandler'
+TBLPROPERTIES ("druid.datasource" = "wikipedia")
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@druid_table_2
+PREHOOK: query: DESCRIBE FORMATTED druid_table_2
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@druid_table_2
+POSTHOOK: query: DESCRIBE FORMATTED druid_table_2
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@druid_table_2
+# col_name data_type comment
+
+__time timestamp from deserializer
+robot string from deserializer
+namespace string from deserializer
+anonymous string from deserializer
+unpatrolled string from deserializer
+page string from deserializer
+language string from deserializer
+newpage string from deserializer
+user string from deserializer
+count float from deserializer
+added float from deserializer
+delta float from deserializer
+variation float from deserializer
+deleted float from deserializer
+
+# Detailed Table Information
+Database: default
+#### A masked pattern was here ####
+Retention: 0
+#### A masked pattern was here ####
+Table Type: EXTERNAL_TABLE
+Table Parameters:
+ COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"}
+ EXTERNAL TRUE
+ druid.datasource wikipedia
+ numFiles 0
+ numRows 0
+ rawDataSize 0
+ storage_handler org.apache.hadoop.hive.druid.QTestDruidStorageHandler
+ totalSize 0
+#### A masked pattern was here ####
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.druid.QTestDruidSerDe
+InputFormat: null
+OutputFormat: null
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
+PREHOOK: query: DROP TABLE druid_table_2
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@druid_table_2
+PREHOOK: Output: default@druid_table_2
+POSTHOOK: query: DROP TABLE druid_table_2
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@druid_table_2
+POSTHOOK: Output: default@druid_table_2
+PREHOOK: query: DROP TABLE druid_table_1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@druid_table_1
+PREHOOK: Output: default@druid_table_1
+POSTHOOK: query: DROP TABLE druid_table_1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@druid_table_1
+POSTHOOK: Output: default@druid_table_1
[6/6] hive git commit: HIVE-14217: Druid integration (Jesus Camacho
Rodriguez, reviewed by Ashutosh Chauhan)
Posted by jc...@apache.org.
HIVE-14217: Druid integration (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)
Close apache/hive#98
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/58d1befa
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/58d1befa
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/58d1befa
Branch: refs/heads/master
Commit: 58d1befa2131254b53122b3573189ac1c5022217
Parents: 63fdb51
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Fri Aug 12 12:55:46 2016 +0100
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Thu Sep 8 09:42:26 2016 +0100
----------------------------------------------------------------------
.../org/apache/hadoop/hive/conf/Constants.java | 10 +-
.../org/apache/hadoop/hive/conf/HiveConf.java | 11 +
druid-handler/pom.xml | 201 ++++
.../hadoop/hive/druid/DruidStorageHandler.java | 109 ++
.../hive/druid/DruidStorageHandlerUtils.java | 90 ++
.../hive/druid/HiveDruidOutputFormat.java | 55 +
.../druid/HiveDruidQueryBasedInputFormat.java | 369 ++++++
.../hadoop/hive/druid/HiveDruidSplit.java | 83 ++
.../serde/DruidGroupByQueryRecordReader.java | 199 ++++
.../druid/serde/DruidQueryRecordReader.java | 142 +++
.../serde/DruidSelectQueryRecordReader.java | 106 ++
.../hadoop/hive/druid/serde/DruidSerDe.java | 343 ++++++
.../hive/druid/serde/DruidSerDeUtils.java | 83 ++
.../serde/DruidTimeseriesQueryRecordReader.java | 93 ++
.../druid/serde/DruidTopNQueryRecordReader.java | 106 ++
.../hadoop/hive/druid/serde/DruidWritable.java | 81 ++
.../hadoop/hive/druid/QTestDruidSerDe.java | 88 ++
.../hive/druid/QTestDruidStorageHandler.java | 34 +
.../hadoop/hive/druid/TestDruidSerDe.java | 576 ++++++++++
.../TestHiveDruidQueryBasedInputFormat.java | 101 ++
itests/qtest/pom.xml | 13 +
packaging/pom.xml | 5 +
pom.xml | 2 +
.../org/apache/hadoop/hive/ql/exec/DDLTask.java | 8 +-
.../hadoop/hive/ql/exec/FunctionRegistry.java | 22 +-
.../optimizer/calcite/HivePlannerContext.java | 17 +-
.../calcite/druid/DruidIntervalUtils.java | 466 ++++++++
.../ql/optimizer/calcite/druid/DruidQuery.java | 1053 ++++++++++++++++++
.../optimizer/calcite/druid/DruidQueryType.java | 42 +
.../ql/optimizer/calcite/druid/DruidRules.java | 591 ++++++++++
.../ql/optimizer/calcite/druid/DruidSchema.java | 51 +
.../ql/optimizer/calcite/druid/DruidTable.java | 121 ++
.../optimizer/calcite/druid/HiveDruidConf.java | 33 +
.../functions/HiveSqlCountAggFunction.java | 2 +-
.../functions/HiveSqlMinMaxAggFunction.java | 2 +-
.../functions/HiveSqlSumAggFunction.java | 2 +-
.../reloperators/HiveDateGranularity.java | 54 +
.../rules/HiveProjectSortTransposeRule.java | 5 +
.../rules/HiveSortProjectTransposeRule.java | 5 +
.../calcite/translator/ASTBuilder.java | 38 +-
.../calcite/translator/ASTConverter.java | 9 +-
.../translator/SqlFunctionConverter.java | 23 +-
.../hadoop/hive/ql/parse/CalcitePlanner.java | 119 +-
.../hadoop/hive/ql/parse/SemanticAnalyzer.java | 22 +-
.../hadoop/hive/ql/plan/CreateTableDesc.java | 8 +-
.../hadoop/hive/ql/plan/TableScanDesc.java | 7 +
.../apache/hadoop/hive/ql/udf/UDFDateFloor.java | 506 +++++++++
.../hadoop/hive/ql/udf/UDFDateFloorDay.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorHour.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorMinute.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorMonth.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorQuarter.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorSecond.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorWeek.java | 39 +
.../hadoop/hive/ql/udf/UDFDateFloorYear.java | 39 +
.../calcite/TestCBORuleFiredOnlyOnce.java | 2 +-
.../ql/udf/TestUDFDateFormatGranularity.java | 85 ++
.../test/queries/clientnegative/druid_address.q | 5 +
.../test/queries/clientnegative/druid_buckets.q | 6 +
.../queries/clientnegative/druid_datasource.q | 3 +
.../queries/clientnegative/druid_external.q | 5 +
.../queries/clientnegative/druid_location.q | 6 +
.../queries/clientnegative/druid_partitions.q | 6 +
.../test/queries/clientpositive/druid_basic1.q | 18 +
.../test/queries/clientpositive/druid_basic2.q | 52 +
.../queries/clientpositive/druid_intervals.q | 67 ++
.../queries/clientpositive/druid_timeseries.q | 94 ++
ql/src/test/queries/clientpositive/druid_topn.q | 75 ++
.../results/clientnegative/druid_address.q.out | 7 +
.../results/clientnegative/druid_buckets.q.out | 8 +
.../clientnegative/druid_datasource.q.out | 7 +
.../results/clientnegative/druid_external.q.out | 7 +
.../results/clientnegative/druid_location.q.out | 9 +
.../clientnegative/druid_partitions.q.out | 8 +
.../results/clientpositive/create_view.q.out | 2 +
.../results/clientpositive/druid_basic1.q.out | 142 +++
.../results/clientpositive/druid_basic2.q.out | 533 +++++++++
.../clientpositive/druid_intervals.q.out | 398 +++++++
.../clientpositive/druid_timeseries.q.out | 566 ++++++++++
.../results/clientpositive/druid_topn.q.out | 419 +++++++
.../results/clientpositive/explain_ddl.q.out | 2 +
.../clientpositive/explain_logical.q.out | 16 +
.../test/results/clientpositive/join_view.q.out | 4 +
.../clientpositive/llap/explainuser_1.q.out | 2 +-
.../test/results/clientpositive/masking_2.q.out | 14 +
.../test/results/clientpositive/masking_6.q.out | 8 +
.../test/results/clientpositive/masking_7.q.out | 8 +
.../clientpositive/serde_user_properties.q.out | 4 +
.../results/clientpositive/show_functions.q.out | 9 +
.../clientpositive/spark/join_view.q.out | 4 +
.../results/clientpositive/subquery_notin.q.out | 6 +
.../results/clientpositive/subquery_views.q.out | 4 +
92 files changed, 8969 insertions(+), 60 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/common/src/java/org/apache/hadoop/hive/conf/Constants.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/Constants.java b/common/src/java/org/apache/hadoop/hive/conf/Constants.java
index 00ec8c0..77c6aa5 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/Constants.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/Constants.java
@@ -15,8 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
-
package org.apache.hadoop.hive.conf;
public class Constants {
@@ -24,4 +22,12 @@ public class Constants {
public static final String LLAP_LOGGER_NAME_QUERY_ROUTING = "query-routing";
public static final String LLAP_LOGGER_NAME_CONSOLE = "console";
public static final String LLAP_LOGGER_NAME_RFA = "RFA";
+
+ /* Constants for Druid storage handler */
+ public static final String DRUID_HIVE_STORAGE_HANDLER_ID =
+ "org.apache.hadoop.hive.druid.DruidStorageHandler";
+ public static final String DRUID_DATA_SOURCE = "druid.datasource";
+ public static final String DRUID_QUERY_JSON = "druid.query.json";
+ public static final String DRUID_QUERY_TYPE = "druid.query.type";
+ public static final String DRUID_QUERY_FETCH = "druid.query.fetch";
}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 13cfdf1..d6944ee 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1884,6 +1884,17 @@ public class HiveConf extends Configuration {
WRITE_SET_REAPER_INTERVAL("hive.writeset.reaper.interval", "60s",
new TimeValidator(TimeUnit.MILLISECONDS), "Frequency of WriteSet reaper runs"),
+ // For Druid storage handler
+ HIVE_DRUID_BROKER_DEFAULT_ADDRESS("hive.druid.broker.address.default", "localhost:8082",
+ "Address of the Druid broker. If we are querying Druid from Hive, this address needs to be\n" +
+ "declared"),
+ HIVE_DRUID_SELECT_THRESHOLD("hive.druid.select.threshold", 10000,
+ "When we can split a Select query, this is the maximum number of rows that we try to retrieve\n" +
+ "per query. In order to do that, we obtain the estimated size for the complete result. If the\n" +
+ "number of records of the query results is larger than this threshold, we split the query in\n" +
+ "total number of rows/threshold parts across the time dimension. Note that we assume the\n" +
+ "records to be split uniformly across the time dimension"),
+
// For HBase storage handler
HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true,
"Whether writes to HBase should be forced to the write-ahead log. \n" +
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/pom.xml
----------------------------------------------------------------------
diff --git a/druid-handler/pom.xml b/druid-handler/pom.xml
new file mode 100644
index 0000000..2173cdc
--- /dev/null
+++ b/druid-handler/pom.xml
@@ -0,0 +1,201 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed under the Apache License, Version 2.0 (the "License");
+ you may not use this file except in compliance with the License.
+ You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive</artifactId>
+ <version>2.2.0-SNAPSHOT</version>
+ <relativePath>../pom.xml</relativePath>
+ </parent>
+
+ <artifactId>hive-druid-handler</artifactId>
+ <packaging>jar</packaging>
+ <name>Hive Druid Handler</name>
+
+ <properties>
+ <hive.path.to.root>..</hive.path.to.root>
+ </properties>
+
+ <dependencies>
+ <!-- dependencies are always listed in sorted order by groupId, artifectId -->
+ <!-- intra-project -->
+ <dependency>
+ <groupId>org.apache.hive</groupId>
+ <artifactId>hive-exec</artifactId>
+ <version>${project.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty-all</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <!-- inter-project -->
+ <dependency>
+ <groupId>commons-lang</groupId>
+ <artifactId>commons-lang</artifactId>
+ <version>${commons-lang.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <version>${hadoop.version}</version>
+ <optional>true</optional>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-log4j12</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>commmons-logging</groupId>
+ <artifactId>commons-logging</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <version>${hadoop.version}</version>
+ <optional>true</optional>
+ <exclusions>
+ <exclusion>
+ <groupId>io.netty</groupId>
+ <artifactId>netty</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>io.druid</groupId>
+ <artifactId>druid-processing</artifactId>
+ <version>${druid.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-core</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-annotations</artifactId>
+ </exclusion>
+ <exclusion>
+ <groupId>com.fasterxml.jackson.core</groupId>
+ <artifactId>jackson-databind</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+
+ <!-- test inter-project -->
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>${junit.version}</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <sourceDirectory>${basedir}/src/java</sourceDirectory>
+ <testSourceDirectory>${basedir}/src/test</testSourceDirectory>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <executions>
+ <execution>
+ <goals>
+ <goal>test-jar</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>${maven.shade.plugin.version}</version>
+ <executions>
+ <!-- we need to shade netty, as there is a conflict between versions
+ used by Hadoop (3.6.2.Final) and Druid (3.10.4.Final) -->
+ <!-- we need to shade jackson, as there is a conflict between versions
+ used by Hive (2.4.2) and Druid (2.4.6) -->
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <shadeTestJar>true</shadeTestJar>
+ <createDependencyReducedPom>false</createDependencyReducedPom>
+ <relocations>
+ <relocation>
+ <pattern>io.druid</pattern>
+ <shadedPattern>org.apache.hive.druid.io.druid</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.metamx.emitter</pattern>
+ <shadedPattern>org.apache.hive.druid.com.metamx.emitter</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.metamx.http.client</pattern>
+ <shadedPattern>org.apache.hive.druid.com.metamx.http.client</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>io.netty</pattern>
+ <shadedPattern>org.apache.hive.druid.io.netty</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>org.jboss.netty</pattern>
+ <shadedPattern>org.apache.hive.druid.org.jboss.netty</shadedPattern>
+ </relocation>
+ <relocation>
+ <pattern>com.fasterxml.jackson</pattern>
+ <shadedPattern>org.apache.hive.druid.com.fasterxml.jackson</shadedPattern>
+ </relocation>
+ </relocations>
+ <artifactSet>
+ <includes>
+ <include>io.druid:*</include>
+ <include>com.metamx:emitter:*</include>
+ <include>com.metamx:http-client:*</include>
+ <include>io.netty:*</include>
+ <include>com.fasterxml.jackson.core:*</include>
+ <include>com.fasterxml.jackson.datatype:*</include>
+ <include>com.fasterxml.jackson.dataformat:*</include>
+ </includes>
+ </artifactSet>
+ <filters>
+ <filter>
+ <artifact>*:*</artifact>
+ <excludes>
+ <exclude>META-INF/*.SF</exclude>
+ <exclude>META-INF/*.DSA</exclude>
+ <exclude>META-INF/*.RSA</exclude>
+ </excludes>
+ </filter>
+ </filters>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+
+</project>
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
new file mode 100644
index 0000000..ac03099
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java
@@ -0,0 +1,109 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.druid.serde.DruidSerDe;
+import org.apache.hadoop.hive.metastore.HiveMetaHook;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler;
+import org.apache.hadoop.hive.serde2.SerDe;
+import org.apache.hadoop.mapred.InputFormat;
+import org.apache.hadoop.mapred.OutputFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DruidStorageHandler provides a HiveStorageHandler implementation for Druid.
+ */
+@SuppressWarnings({"deprecation","rawtypes"})
+public class DruidStorageHandler extends DefaultStorageHandler implements HiveMetaHook {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(DruidStorageHandler.class);
+
+ @Override
+ public Class<? extends InputFormat> getInputFormatClass() {
+ return HiveDruidQueryBasedInputFormat.class;
+ }
+
+ @Override
+ public Class<? extends OutputFormat> getOutputFormatClass() {
+ return HiveDruidOutputFormat.class;
+ }
+
+ @Override
+ public Class<? extends SerDe> getSerDeClass() {
+ return DruidSerDe.class;
+ }
+
+ @Override
+ public HiveMetaHook getMetaHook() {
+ return this;
+ }
+
+ @Override
+ public void preCreateTable(Table table) throws MetaException {
+ // Do safety checks
+ if (!MetaStoreUtils.isExternalTable(table)) {
+ throw new MetaException("Table in Druid needs to be declared as EXTERNAL");
+ }
+ if (!StringUtils.isEmpty(table.getSd().getLocation())) {
+ throw new MetaException("LOCATION may not be specified for Druid");
+ }
+ if (table.getPartitionKeysSize() != 0) {
+ throw new MetaException("PARTITIONED BY may not be specified for Druid");
+ }
+ if (table.getSd().getBucketColsSize() != 0) {
+ throw new MetaException("CLUSTERED BY may not be specified for Druid");
+ }
+ }
+
+ @Override
+ public void rollbackCreateTable(Table table) throws MetaException {
+ // Nothing to do
+ }
+
+ @Override
+ public void commitCreateTable(Table table) throws MetaException {
+ // Nothing to do
+ }
+
+ @Override
+ public void preDropTable(Table table) throws MetaException {
+ // Nothing to do
+ }
+
+ @Override
+ public void rollbackDropTable(Table table) throws MetaException {
+ // Nothing to do
+ }
+
+ @Override
+ public void commitDropTable(Table table, boolean deleteData) throws MetaException {
+ // Nothing to do
+ }
+
+ @Override
+ public String toString() {
+ return Constants.DRUID_HIVE_STORAGE_HANDLER_ID;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
new file mode 100644
index 0000000..c6b8024
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandlerUtils.java
@@ -0,0 +1,90 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.concurrent.ExecutionException;
+
+import org.jboss.netty.handler.codec.http.HttpHeaders;
+import org.jboss.netty.handler.codec.http.HttpMethod;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.smile.SmileFactory;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.Request;
+import com.metamx.http.client.response.InputStreamResponseHandler;
+
+import io.druid.jackson.DefaultObjectMapper;
+import io.druid.query.BaseQuery;
+
+/**
+ * Utils class for Druid storage handler.
+ */
+public final class DruidStorageHandlerUtils {
+
+ private static final String SMILE_CONTENT_TYPE = "application/x-jackson-smile";
+
+ /**
+ * Mapper to use to serialize/deserialize Druid objects (JSON)
+ */
+ public static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
+
+ /**
+ * Mapper to use to serialize/deserialize Druid objects (SMILE)
+ */
+ public static final ObjectMapper SMILE_MAPPER = new DefaultObjectMapper(new SmileFactory());
+
+ /**
+ * Method that creates a request for Druid JSON query (using SMILE).
+ * @param mapper
+ * @param address
+ * @param query
+ * @return
+ * @throws IOException
+ */
+ public static Request createRequest(String address, BaseQuery<?> query)
+ throws IOException {
+ return new Request(HttpMethod.POST, new URL(String.format("%s/druid/v2/", "http://" + address)))
+ .setContent(SMILE_MAPPER.writeValueAsBytes(query))
+ .setHeader(HttpHeaders.Names.CONTENT_TYPE, SMILE_CONTENT_TYPE);
+ }
+
+ /**
+ * Method that submits a request to an Http address and retrieves the result.
+ * The caller is responsible for closing the stream once it finishes consuming it.
+ * @param client
+ * @param request
+ * @return
+ * @throws IOException
+ */
+ public static InputStream submitRequest(HttpClient client, Request request)
+ throws IOException {
+ InputStream response;
+ try {
+ response = client.go(request, new InputStreamResponseHandler()).get();
+ } catch (ExecutionException e) {
+ throw new IOException(e.getCause());
+ } catch (InterruptedException e) {
+ throw new IOException(e.getCause());
+ }
+ return response;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
new file mode 100644
index 0000000..45e31d6
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidOutputFormat.java
@@ -0,0 +1,55 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.io.HiveOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordWriter;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Place holder for Druid output format. Currently not implemented.
+ */
+@SuppressWarnings("rawtypes")
+public class HiveDruidOutputFormat implements HiveOutputFormat {
+
+ @Override
+ public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name,
+ Progressable progress) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public FileSinkOperator.RecordWriter getHiveRecordWriter(JobConf jc, Path finalOutPath,
+ Class valueClass, boolean isCompressed, Properties tableProperties, Progressable progress)
+ throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
new file mode 100644
index 0000000..3df1452
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidQueryBasedInputFormat.java
@@ -0,0 +1,369 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.lang3.StringEscapeUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.druid.serde.DruidGroupByQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidSelectQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTimeseriesQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidTopNQueryRecordReader;
+import org.apache.hadoop.hive.druid.serde.DruidWritable;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidIntervalUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.joda.time.Interval;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.Druids.SelectQueryBuilder;
+import io.druid.query.Druids.TimeBoundaryQueryBuilder;
+import io.druid.query.Query;
+import io.druid.query.Result;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import io.druid.query.select.PagingSpec;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.spec.MultipleIntervalSegmentSpec;
+import io.druid.query.timeboundary.TimeBoundaryQuery;
+import io.druid.query.timeboundary.TimeBoundaryResultValue;
+
+/**
+ * Druid query based input format.
+ *
+ * Given a query and the Druid broker address, it will send it, and retrieve
+ * and parse the results.
+ */
+public class HiveDruidQueryBasedInputFormat extends InputFormat<NullWritable, DruidWritable>
+ implements org.apache.hadoop.mapred.InputFormat<NullWritable, DruidWritable> {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(HiveDruidQueryBasedInputFormat.class);
+
+ @Override
+ public org.apache.hadoop.mapred.InputSplit[] getSplits(JobConf job, int numSplits)
+ throws IOException {
+ return getInputSplits(job);
+ }
+
+ @Override
+ public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
+ return Arrays.<InputSplit> asList(getInputSplits(context.getConfiguration()));
+ }
+
+ @SuppressWarnings("deprecation")
+ private HiveDruidSplit[] getInputSplits(Configuration conf) throws IOException {
+ String address = HiveConf.getVar(conf,
+ HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+ if (StringUtils.isEmpty(address)) {
+ throw new IOException("Druid broker address not specified in configuration");
+ }
+ String druidQuery = StringEscapeUtils.unescapeJava(conf.get(Constants.DRUID_QUERY_JSON));
+ String druidQueryType;
+ if (StringUtils.isEmpty(druidQuery)) {
+ // Empty, maybe because CBO did not run; we fall back to
+ // full Select query
+ if (LOG.isWarnEnabled()) {
+ LOG.warn("Druid query is empty; creating Select query");
+ }
+ String dataSource = conf.get(Constants.DRUID_DATA_SOURCE);
+ if (dataSource == null) {
+ throw new IOException("Druid data source cannot be empty");
+ }
+ druidQuery = createSelectStarQuery(address, dataSource);
+ druidQueryType = Query.SELECT;
+ } else {
+ druidQueryType = conf.get(Constants.DRUID_QUERY_TYPE);
+ if (druidQueryType == null) {
+ throw new IOException("Druid query type not recognized");
+ }
+ }
+
+ // hive depends on FileSplits
+ Job job = new Job(conf);
+ JobContext jobContext = ShimLoader.getHadoopShims().newJobContext(job);
+ Path [] paths = FileInputFormat.getInputPaths(jobContext);
+
+ switch (druidQueryType) {
+ case Query.TIMESERIES:
+ case Query.TOPN:
+ case Query.GROUP_BY:
+ return new HiveDruidSplit[] { new HiveDruidSplit(address, druidQuery, paths[0]) };
+ case Query.SELECT:
+ return splitSelectQuery(conf, address, druidQuery, paths[0]);
+ default:
+ throw new IOException("Druid query type not recognized");
+ }
+ }
+
+ private static String createSelectStarQuery(String address, String dataSource) throws IOException {
+ // Create Select query
+ SelectQueryBuilder builder = new Druids.SelectQueryBuilder();
+ builder.dataSource(dataSource);
+ builder.intervals(Arrays.asList(DruidTable.DEFAULT_INTERVAL));
+ builder.pagingSpec(PagingSpec.newSpec(1));
+ Map<String, Object> context = new HashMap<>();
+ context.put(Constants.DRUID_QUERY_FETCH, false);
+ builder.context(context);
+ return DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(builder.build());
+ }
+
+ /* Method that splits Select query depending on the threshold so read can be
+ * parallelized */
+ private static HiveDruidSplit[] splitSelectQuery(Configuration conf, String address,
+ String druidQuery, Path dummyPath) throws IOException {
+ final int selectThreshold = (int) HiveConf.getIntVar(
+ conf, HiveConf.ConfVars.HIVE_DRUID_SELECT_THRESHOLD);
+
+ SelectQuery query;
+ try {
+ query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, SelectQuery.class);
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+
+ final boolean isFetch = query.getContextBoolean(Constants.DRUID_QUERY_FETCH, false);
+ if (isFetch) {
+ // If it has a limit, we use it and we do not split the query
+ return new HiveDruidSplit[] { new HiveDruidSplit(
+ address, DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+ }
+
+ // We do not have the number of rows, thus we need to execute a
+ // Segment Metadata query to obtain number of rows
+ SegmentMetadataQueryBuilder metadataBuilder = new Druids.SegmentMetadataQueryBuilder();
+ metadataBuilder.dataSource(query.getDataSource());
+ metadataBuilder.intervals(query.getIntervals());
+ metadataBuilder.merge(true);
+ metadataBuilder.analysisTypes();
+ SegmentMetadataQuery metadataQuery = metadataBuilder.build();
+
+ HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+ InputStream response;
+ try {
+ response = DruidStorageHandlerUtils.submitRequest(client,
+ DruidStorageHandlerUtils.createRequest(address, metadataQuery));
+ } catch (Exception e) {
+ throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+
+ // Retrieve results
+ List<SegmentAnalysis> metadataList;
+ try {
+ metadataList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+ new TypeReference<List<SegmentAnalysis>>() {});
+ } catch (Exception e) {
+ response.close();
+ throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+ if (metadataList == null || metadataList.isEmpty()) {
+ throw new IOException("Connected to Druid but could not retrieve datasource information");
+ }
+ if (metadataList.size() != 1) {
+ throw new IOException("Information about segments should have been merged");
+ }
+
+ final long numRows = metadataList.get(0).getNumRows();
+
+ query = query.withPagingSpec(PagingSpec.newSpec(selectThreshold));
+ if (numRows <= selectThreshold) {
+ // We are not going to split it
+ return new HiveDruidSplit[] { new HiveDruidSplit(address,
+ DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(query), dummyPath) };
+ }
+
+ // If the query does not specify a timestamp, we obtain the total time using
+ // a Time Boundary query. Then, we use the information to split the query
+ // following the Select threshold configuration property
+ final List<Interval> intervals = new ArrayList<>();
+ if (query.getIntervals().size() == 1 &&
+ query.getIntervals().get(0).equals(DruidTable.DEFAULT_INTERVAL)) {
+ // Default max and min, we should execute a time boundary query to get a
+ // more precise range
+ TimeBoundaryQueryBuilder timeBuilder = new Druids.TimeBoundaryQueryBuilder();
+ timeBuilder.dataSource(query.getDataSource());
+ TimeBoundaryQuery timeQuery = timeBuilder.build();
+
+ try {
+ response = DruidStorageHandlerUtils.submitRequest(client,
+ DruidStorageHandlerUtils.createRequest(address, timeQuery));
+ } catch (Exception e) {
+ throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+
+ // Retrieve results
+ List<Result<TimeBoundaryResultValue>> timeList;
+ try {
+ timeList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+ new TypeReference<List<Result<TimeBoundaryResultValue>>>() {});
+ } catch (Exception e) {
+ response.close();
+ throw new IOException(org.apache.hadoop.util.StringUtils.stringifyException(e));
+ }
+ if (timeList == null || timeList.isEmpty()) {
+ throw new IOException("Connected to Druid but could not retrieve time boundary information");
+ }
+ if (timeList.size() != 1) {
+ throw new IOException("We should obtain a single time boundary");
+ }
+
+ intervals.add(new Interval(timeList.get(0).getValue().getMinTime().getMillis(),
+ timeList.get(0).getValue().getMaxTime().getMillis()));
+ } else {
+ intervals.addAll(query.getIntervals());
+ }
+
+ // Create (numRows/default threshold) input splits
+ int numSplits = (int) Math.ceil((double) numRows / selectThreshold);
+ List<List<Interval>> newIntervals = createSplitsIntervals(intervals, numSplits);
+ HiveDruidSplit[] splits = new HiveDruidSplit[numSplits];
+ for (int i = 0; i < numSplits; i++) {
+ // Create partial Select query
+ final SelectQuery partialQuery = query.withQuerySegmentSpec(
+ new MultipleIntervalSegmentSpec(newIntervals.get(i)));
+ splits[i] = new HiveDruidSplit(address,
+ DruidStorageHandlerUtils.JSON_MAPPER.writeValueAsString(partialQuery), dummyPath);
+ }
+ return splits;
+ }
+
+ private static List<List<Interval>> createSplitsIntervals(List<Interval> intervals, int numSplits) {
+ final long totalTime = DruidIntervalUtils.extractTotalTime(intervals);
+ long startTime = intervals.get(0).getStartMillis();
+ long endTime = startTime;
+ long currTime = 0;
+ List<List<Interval>> newIntervals = new ArrayList<>();
+ for (int i = 0, posIntervals = 0; i < numSplits; i++) {
+ final long rangeSize = Math.round( (double) (totalTime * (i + 1)) / numSplits) -
+ Math.round( (double) (totalTime * i) / numSplits);
+ // Create the new interval(s)
+ List<Interval> currentIntervals = new ArrayList<>();
+ while (posIntervals < intervals.size()) {
+ final Interval interval = intervals.get(posIntervals);
+ final long expectedRange = rangeSize - currTime;
+ if (interval.getEndMillis() - startTime >= expectedRange) {
+ endTime = startTime + expectedRange;
+ currentIntervals.add(new Interval(startTime, endTime));
+ startTime = endTime;
+ currTime = 0;
+ break;
+ }
+ endTime = interval.getEndMillis();
+ currentIntervals.add(new Interval(startTime, endTime));
+ currTime += (endTime - startTime);
+ startTime = intervals.get(++posIntervals).getStartMillis();
+ }
+ newIntervals.add(currentIntervals);
+ }
+ assert endTime == intervals.get(intervals.size()-1).getEndMillis();
+ return newIntervals;
+ }
+
+ @Override
+ public org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> getRecordReader(
+ org.apache.hadoop.mapred.InputSplit split, JobConf job, Reporter reporter)
+ throws IOException {
+ // We need to provide a different record reader for every type of Druid query.
+ // The reason is that Druid results format is different for each type.
+ final DruidQueryRecordReader<?,?> reader;
+ final String druidQueryType = job.get(Constants.DRUID_QUERY_TYPE);
+ if (druidQueryType == null) {
+ reader = new DruidSelectQueryRecordReader(); // By default
+ reader.initialize((HiveDruidSplit)split, job);
+ return reader;
+ }
+ switch (druidQueryType) {
+ case Query.TIMESERIES:
+ reader = new DruidTimeseriesQueryRecordReader();
+ break;
+ case Query.TOPN:
+ reader = new DruidTopNQueryRecordReader();
+ break;
+ case Query.GROUP_BY:
+ reader = new DruidGroupByQueryRecordReader();
+ break;
+ case Query.SELECT:
+ reader = new DruidSelectQueryRecordReader();
+ break;
+ default:
+ throw new IOException("Druid query type not recognized");
+ }
+ reader.initialize((HiveDruidSplit)split, job);
+ return reader;
+ }
+
+ @Override
+ public RecordReader<NullWritable, DruidWritable> createRecordReader(InputSplit split,
+ TaskAttemptContext context) throws IOException, InterruptedException {
+ // We need to provide a different record reader for every type of Druid query.
+ // The reason is that Druid results format is different for each type.
+ final String druidQueryType = context.getConfiguration().get(Constants.DRUID_QUERY_TYPE);
+ if (druidQueryType == null) {
+ return new DruidSelectQueryRecordReader(); // By default
+ }
+ final DruidQueryRecordReader<?,?> reader;
+ switch (druidQueryType) {
+ case Query.TIMESERIES:
+ reader = new DruidTimeseriesQueryRecordReader();
+ break;
+ case Query.TOPN:
+ reader = new DruidTopNQueryRecordReader();
+ break;
+ case Query.GROUP_BY:
+ reader = new DruidGroupByQueryRecordReader();
+ break;
+ case Query.SELECT:
+ reader = new DruidSelectQueryRecordReader();
+ break;
+ default:
+ throw new IOException("Druid query type not recognized");
+ }
+ return reader;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
new file mode 100644
index 0000000..3fba5d0
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/HiveDruidSplit.java
@@ -0,0 +1,83 @@
+/**
+ * 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.hadoop.hive.druid;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
+
+/**
+ * Druid split. Its purpose is to trigger query execution in Druid.
+ */
+public class HiveDruidSplit extends FileSplit implements org.apache.hadoop.mapred.InputSplit {
+
+ private String address;
+ private String druidQuery;
+
+ // required for deserialization
+ public HiveDruidSplit() {
+ super((Path) null, 0, 0, (String[]) null);
+ }
+
+ public HiveDruidSplit(String address, String druidQuery, Path dummyPath) {
+ super(dummyPath, 0, 0, (String[]) null);
+ this.address = address;
+ this.druidQuery = druidQuery;
+ }
+
+ @Override
+ public void write(DataOutput out) throws IOException {
+ super.write(out);
+ out.writeUTF(address);
+ out.writeUTF(druidQuery);
+ }
+
+ @Override
+ public void readFields(DataInput in) throws IOException {
+ super.readFields(in);
+ address = in.readUTF();
+ druidQuery = in.readUTF();
+ }
+
+ @Override
+ public long getLength() {
+ return 0L;
+ }
+
+ @Override
+ public String[] getLocations() {
+ return new String[] {""} ;
+ }
+
+ public String getAddress() {
+ return address;
+ }
+
+ public String getDruidQuery() {
+ return druidQuery;
+ }
+
+ @Override
+ public String toString() {
+ return "HiveDruidSplit{" + address + ", " + druidQuery + "}";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
new file mode 100644
index 0000000..226060f
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidGroupByQueryRecordReader.java
@@ -0,0 +1,199 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.data.input.Row;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.PostAggregator;
+import io.druid.query.dimension.DimensionSpec;
+import io.druid.query.groupby.GroupByQuery;
+
+/**
+ * Record reader for results for Druid GroupByQuery.
+ */
+public class DruidGroupByQueryRecordReader
+ extends DruidQueryRecordReader<GroupByQuery, Row> {
+
+ private Row current;
+ private int[] indexes = new int[0];
+ // Row objects returned by GroupByQuery have different access paths depending on
+ // whether the result for the metric is a Float or a Long, thus we keep track
+ // using these converters
+ private Extract[] extractors;
+
+ @Override
+ public void initialize(InputSplit split, Configuration conf) throws IOException {
+ super.initialize(split, conf);
+ initExtractors();
+ }
+
+ @Override
+ protected GroupByQuery createQuery(String content) throws IOException {
+ return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, GroupByQuery.class);
+ }
+
+ @Override
+ protected List<Row> createResultsList(InputStream content) throws IOException {
+ return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+ new TypeReference<List<Row>>(){});
+ }
+
+ private void initExtractors() throws IOException {
+ extractors = new Extract[query.getAggregatorSpecs().size() + query.getPostAggregatorSpecs().size()];
+ int counter = 0;
+ for (int i = 0; i < query.getAggregatorSpecs().size(); i++, counter++) {
+ AggregatorFactory af = query.getAggregatorSpecs().get(i);
+ switch (af.getTypeName().toUpperCase()) {
+ case DruidSerDeUtils.FLOAT_TYPE:
+ extractors[counter] = Extract.FLOAT;
+ break;
+ case DruidSerDeUtils.LONG_TYPE:
+ extractors[counter] = Extract.LONG;
+ break;
+ default:
+ throw new IOException("Type not supported");
+ }
+ }
+ for (int i = 0; i < query.getPostAggregatorSpecs().size(); i++, counter++) {
+ extractors[counter] = Extract.FLOAT;
+ }
+ }
+
+ @Override
+ public boolean nextKeyValue() {
+ // Refresh indexes
+ for (int i = indexes.length - 1; i >= 0; i--) {
+ if (indexes[i] > 0) {
+ indexes[i]--;
+ for (int j = i + 1; j < indexes.length; j++) {
+ indexes[j] = current.getDimension(
+ query.getDimensions().get(j).getDimension()).size() - 1;
+ }
+ return true;
+ }
+ }
+ // Results
+ if (results.hasNext()) {
+ current = results.next();
+ indexes = new int[query.getDimensions().size()];
+ for (int i=0; i < query.getDimensions().size(); i++) {
+ DimensionSpec ds = query.getDimensions().get(i);
+ indexes[i] = current.getDimension(ds.getDimension()).size() - 1;
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public NullWritable getCurrentKey() throws IOException, InterruptedException {
+ return NullWritable.get();
+ }
+
+ @Override
+ public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+ // Create new value
+ DruidWritable value = new DruidWritable();
+ // 1) The timestamp column
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ // 2) The dimension columns
+ for (int i=0; i < query.getDimensions().size(); i++) {
+ DimensionSpec ds = query.getDimensions().get(i);
+ List<String> dims = current.getDimension(ds.getDimension());
+ int pos = dims.size() - indexes[i] - 1;
+ value.getValue().put(ds.getOutputName(), dims.get(pos));
+ }
+ int counter = 0;
+ // 3) The aggregation columns
+ for (AggregatorFactory af : query.getAggregatorSpecs()) {
+ switch (extractors[counter++]) {
+ case FLOAT:
+ value.getValue().put(af.getName(), current.getFloatMetric(af.getName()));
+ break;
+ case LONG:
+ value.getValue().put(af.getName(), current.getLongMetric(af.getName()));
+ break;
+ }
+ }
+ // 4) The post-aggregation columns
+ for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+ assert extractors[counter++] == Extract.FLOAT;
+ value.getValue().put(pa.getName(), current.getFloatMetric(pa.getName()));
+ }
+ return value;
+ }
+
+ @Override
+ public boolean next(NullWritable key, DruidWritable value) {
+ if (nextKeyValue()) {
+ // Update value
+ value.getValue().clear();
+ // 1) The timestamp column
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ // 2) The dimension columns
+ for (int i=0; i < query.getDimensions().size(); i++) {
+ DimensionSpec ds = query.getDimensions().get(i);
+ List<String> dims = current.getDimension(ds.getDimension());
+ int pos = dims.size() - indexes[i] - 1;
+ value.getValue().put(ds.getOutputName(), dims.get(pos));
+ }
+ int counter = 0;
+ // 3) The aggregation columns
+ for (AggregatorFactory af : query.getAggregatorSpecs()) {
+ switch (extractors[counter++]) {
+ case FLOAT:
+ value.getValue().put(af.getName(), current.getFloatMetric(af.getName()));
+ break;
+ case LONG:
+ value.getValue().put(af.getName(), current.getLongMetric(af.getName()));
+ break;
+ }
+ }
+ // 4) The post-aggregation columns
+ for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+ assert extractors[counter++] == Extract.FLOAT;
+ value.getValue().put(pa.getName(), current.getFloatMetric(pa.getName()));
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return results.hasNext() ? 0 : 1;
+ }
+
+ private enum Extract {
+ FLOAT,
+ LONG
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
new file mode 100644
index 0000000..96bcee8
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidQueryRecordReader.java
@@ -0,0 +1,142 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.druid.HiveDruidSplit;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Iterators;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.BaseQuery;
+
+/**
+ * Base record reader for given a Druid query. This class contains the logic to
+ * send the query to the broker and retrieve the results. The transformation to
+ * emit records needs to be done by the classes that extend the reader.
+ *
+ * The key for each record will be a NullWritable, while the value will be a
+ * DruidWritable containing the timestamp as well as all values resulting from
+ * the query.
+ */
+public abstract class DruidQueryRecordReader<T extends BaseQuery<R>,R extends Comparable<R>>
+ extends RecordReader<NullWritable, DruidWritable>
+ implements org.apache.hadoop.mapred.RecordReader<NullWritable, DruidWritable> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(DruidQueryRecordReader.class);
+
+ /**
+ * Query that Druid executes.
+ */
+ protected T query;
+
+ /**
+ * Query results.
+ */
+ protected Iterator<R> results = Iterators.emptyIterator();
+
+ @Override
+ public void initialize(InputSplit split, TaskAttemptContext context) throws IOException {
+ initialize(split, context.getConfiguration());
+ }
+
+ public void initialize(InputSplit split, Configuration conf) throws IOException {
+ HiveDruidSplit hiveDruidSplit = (HiveDruidSplit) split;
+
+ // Create query
+ query = createQuery(hiveDruidSplit.getDruidQuery());
+
+ // Execute query
+ if (LOG.isInfoEnabled()) {
+ LOG.info("Retrieving from druid using query:\n " + query);
+ }
+
+ HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+ InputStream response = DruidStorageHandlerUtils.submitRequest(client,
+ DruidStorageHandlerUtils.createRequest(hiveDruidSplit.getAddress(), query));
+
+ // Retrieve results
+ List<R> resultsList;
+ try {
+ resultsList = createResultsList(response);
+ } catch (IOException e) {
+ response.close();
+ throw e;
+ }
+ if (resultsList == null || resultsList.isEmpty()) {
+ return;
+ }
+ results = resultsList.iterator();
+ }
+
+ protected abstract T createQuery(String content) throws IOException;
+
+ protected abstract List<R> createResultsList(InputStream content) throws IOException;
+
+ @Override
+ public NullWritable createKey() {
+ return NullWritable.get();
+ }
+
+ @Override
+ public DruidWritable createValue() {
+ return new DruidWritable();
+ }
+
+ @Override
+ public abstract boolean next(NullWritable key, DruidWritable value) throws IOException;
+
+ @Override
+ public long getPos() {
+ return 0;
+ }
+
+ @Override
+ public abstract boolean nextKeyValue() throws IOException;
+
+ @Override
+ public abstract NullWritable getCurrentKey() throws IOException, InterruptedException;
+
+ @Override
+ // TODO: we could generate vector row batches so that vectorized execution may get triggered
+ public abstract DruidWritable getCurrentValue() throws IOException, InterruptedException;
+
+ @Override
+ public abstract float getProgress() throws IOException;
+
+ @Override
+ public void close() {
+ // Nothing to do
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
new file mode 100644
index 0000000..70b493c
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSelectQueryRecordReader.java
@@ -0,0 +1,106 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Iterators;
+
+import io.druid.query.Result;
+import io.druid.query.select.EventHolder;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.select.SelectResultValue;
+
+/**
+ * Record reader for results for Druid SelectQuery.
+ */
+public class DruidSelectQueryRecordReader
+ extends DruidQueryRecordReader<SelectQuery, Result<SelectResultValue>> {
+
+ private Result<SelectResultValue> current;
+ private Iterator<EventHolder> values = Iterators.emptyIterator();
+
+ @Override
+ protected SelectQuery createQuery(String content) throws IOException {
+ return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, SelectQuery.class);
+ }
+
+ @Override
+ protected List<Result<SelectResultValue>> createResultsList(InputStream content) throws IOException {
+ return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+ new TypeReference<List<Result<SelectResultValue>>>(){});
+ }
+
+ @Override
+ public boolean nextKeyValue() throws IOException {
+ if (values.hasNext()) {
+ return true;
+ }
+ if (results.hasNext()) {
+ current = results.next();
+ values = current.getValue().getEvents().iterator();
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public NullWritable getCurrentKey() throws IOException, InterruptedException {
+ return NullWritable.get();
+ }
+
+ @Override
+ public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+ // Create new value
+ DruidWritable value = new DruidWritable();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ if (values.hasNext()) {
+ value.getValue().putAll(values.next().getEvent());
+ return value;
+ }
+ return value;
+ }
+
+ @Override
+ public boolean next(NullWritable key, DruidWritable value) throws IOException {
+ if (nextKeyValue()) {
+ // Update value
+ value.getValue().clear();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ if (values.hasNext()) {
+ value.getValue().putAll(values.next().getEvent());
+ }
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public float getProgress() {
+ return results.hasNext() || values.hasNext() ? 0 : 1;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
new file mode 100644
index 0000000..8f53d4a
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDe.java
@@ -0,0 +1,343 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.Constants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.hive.serde2.AbstractSerDe;
+import org.apache.hadoop.hive.serde2.SerDeException;
+import org.apache.hadoop.hive.serde2.SerDeSpec;
+import org.apache.hadoop.hive.serde2.SerDeStats;
+import org.apache.hadoop.hive.serde2.io.TimestampWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.google.common.collect.Lists;
+import com.metamx.common.lifecycle.Lifecycle;
+import com.metamx.http.client.HttpClient;
+import com.metamx.http.client.HttpClientConfig;
+import com.metamx.http.client.HttpClientInit;
+
+import io.druid.query.Druids;
+import io.druid.query.Druids.SegmentMetadataQueryBuilder;
+import io.druid.query.Query;
+import io.druid.query.aggregation.AggregatorFactory;
+import io.druid.query.aggregation.PostAggregator;
+import io.druid.query.dimension.DimensionSpec;
+import io.druid.query.groupby.GroupByQuery;
+import io.druid.query.metadata.metadata.ColumnAnalysis;
+import io.druid.query.metadata.metadata.SegmentAnalysis;
+import io.druid.query.metadata.metadata.SegmentMetadataQuery;
+import io.druid.query.select.SelectQuery;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.topn.TopNQuery;
+
+/**
+ * DruidSerDe that is used to deserialize objects from a Druid data source.
+ */
+@SerDeSpec(schemaProps = {Constants.DRUID_DATA_SOURCE})
+public class DruidSerDe extends AbstractSerDe {
+
+ protected static final Logger LOG = LoggerFactory.getLogger(DruidSerDe.class);
+
+ private String[] columns;
+ private PrimitiveTypeInfo[] types;
+ private ObjectInspector inspector;
+
+
+ @Override
+ public void initialize(Configuration configuration, Properties properties) throws SerDeException {
+ final List<String> columnNames = new ArrayList<>();
+ final List<PrimitiveTypeInfo> columnTypes = new ArrayList<>();
+ List<ObjectInspector> inspectors = new ArrayList<>();
+
+ // Druid query
+ String druidQuery = properties.getProperty(Constants.DRUID_QUERY_JSON);
+ if (druidQuery == null) {
+ // No query. We need to create a Druid Segment Metadata query that retrieves all
+ // columns present in the data source (dimensions and metrics).
+ // Create Segment Metadata Query
+ String dataSource = properties.getProperty(Constants.DRUID_DATA_SOURCE);
+ if (dataSource == null) {
+ throw new SerDeException("Druid data source not specified; use " +
+ Constants.DRUID_DATA_SOURCE + " in table properties");
+ }
+ SegmentMetadataQueryBuilder builder = new Druids.SegmentMetadataQueryBuilder();
+ builder.dataSource(dataSource);
+ builder.merge(true);
+ builder.analysisTypes();
+ SegmentMetadataQuery query = builder.build();
+
+ // Execute query in Druid
+ String address = HiveConf.getVar(configuration,
+ HiveConf.ConfVars.HIVE_DRUID_BROKER_DEFAULT_ADDRESS);
+ if (org.apache.commons.lang3.StringUtils.isEmpty(address)) {
+ throw new SerDeException("Druid broker address not specified in configuration");
+ }
+
+ // Infer schema
+ SegmentAnalysis schemaInfo;
+ try {
+ schemaInfo = submitMetadataRequest(address, query);
+ } catch (IOException e) {
+ throw new SerDeException(e);
+ }
+ for (Entry<String,ColumnAnalysis> columnInfo : schemaInfo.getColumns().entrySet()) {
+ if (columnInfo.getKey().equals(DruidTable.DEFAULT_TIMESTAMP_COLUMN)) {
+ // Special handling for timestamp column
+ columnNames.add(columnInfo.getKey()); // field name
+ PrimitiveTypeInfo type = TypeInfoFactory.timestampTypeInfo; // field type
+ columnTypes.add(type);
+ inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type));
+ continue;
+ }
+ columnNames.add(columnInfo.getKey()); // field name
+ PrimitiveTypeInfo type = DruidSerDeUtils.convertDruidToHiveType(
+ columnInfo.getValue().getType()); // field type
+ columnTypes.add(type);
+ inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(type));
+ }
+ columns = columnNames.toArray(new String[columnNames.size()]);
+ types = columnTypes.toArray(new PrimitiveTypeInfo[columnTypes.size()]);
+ inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors);
+ } else {
+ // Query is specified, we can extract the results schema from the query
+ Query<?> query;
+ try {
+ query = DruidStorageHandlerUtils.JSON_MAPPER.readValue(druidQuery, Query.class);
+ } catch (Exception e) {
+ throw new SerDeException(e);
+ }
+
+ switch (query.getType()) {
+ case Query.TIMESERIES:
+ inferSchema((TimeseriesQuery) query, columnNames, columnTypes);
+ break;
+ case Query.TOPN:
+ inferSchema((TopNQuery) query, columnNames, columnTypes);
+ break;
+ case Query.SELECT:
+ inferSchema((SelectQuery) query, columnNames, columnTypes);
+ break;
+ case Query.GROUP_BY:
+ inferSchema((GroupByQuery) query, columnNames, columnTypes);
+ break;
+ default:
+ throw new SerDeException("Not supported Druid query");
+ }
+
+ columns = new String[columnNames.size()];
+ types = new PrimitiveTypeInfo[columnNames.size()];
+ for (int i = 0; i < columnTypes.size(); ++i) {
+ columns[i] = columnNames.get(i);
+ types[i] = columnTypes.get(i);
+ inspectors.add(PrimitiveObjectInspectorFactory.getPrimitiveWritableObjectInspector(types[i]));
+ }
+ inspector = ObjectInspectorFactory.getStandardStructObjectInspector(columnNames, inspectors);
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("DruidSerDe initialized with\n"
+ + "\t columns: " + columnNames
+ + "\n\t types: " + columnTypes);
+ }
+ }
+
+ /* Submits the request and returns */
+ protected SegmentAnalysis submitMetadataRequest(String address, SegmentMetadataQuery query)
+ throws SerDeException, IOException {
+ HttpClient client = HttpClientInit.createClient(HttpClientConfig.builder().build(), new Lifecycle());
+ InputStream response;
+ try {
+ response = DruidStorageHandlerUtils.submitRequest(client,
+ DruidStorageHandlerUtils.createRequest(address, query));
+ } catch (Exception e) {
+ throw new SerDeException(StringUtils.stringifyException(e));
+ }
+
+ // Retrieve results
+ List<SegmentAnalysis> resultsList;
+ try {
+ resultsList = DruidStorageHandlerUtils.SMILE_MAPPER.readValue(response,
+ new TypeReference<List<SegmentAnalysis>>() {});
+ } catch (Exception e) {
+ response.close();
+ throw new SerDeException(StringUtils.stringifyException(e));
+ }
+ if (resultsList == null || resultsList.isEmpty()) {
+ throw new SerDeException("Connected to Druid but could not retrieve datasource information");
+ }
+ if (resultsList.size() != 1) {
+ throw new SerDeException("Information about segments should have been merged");
+ }
+
+ return resultsList.get(0);
+ }
+
+ /* Timeseries query */
+ private void inferSchema(TimeseriesQuery query, List<String> columnNames,
+ List<PrimitiveTypeInfo> columnTypes) {
+ // Timestamp column
+ columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+ columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+ // Aggregator columns
+ for (AggregatorFactory af : query.getAggregatorSpecs()) {
+ columnNames.add(af.getName());
+ columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+ }
+ // Post-aggregator columns
+ for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+ columnNames.add(pa.getName());
+ columnTypes.add(TypeInfoFactory.floatTypeInfo);
+ }
+ }
+
+ /* TopN query */
+ private void inferSchema(TopNQuery query, List<String> columnNames, List<PrimitiveTypeInfo> columnTypes) {
+ // Timestamp column
+ columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+ columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+ // Dimension column
+ columnNames.add(query.getDimensionSpec().getOutputName());
+ columnTypes.add(TypeInfoFactory.stringTypeInfo);
+ // Aggregator columns
+ for (AggregatorFactory af : query.getAggregatorSpecs()) {
+ columnNames.add(af.getName());
+ columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+ }
+ // Post-aggregator columns
+ for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+ columnNames.add(pa.getName());
+ columnTypes.add(TypeInfoFactory.floatTypeInfo);
+ }
+ }
+
+ /* Select query */
+ private void inferSchema(SelectQuery query, List<String> columnNames,
+ List<PrimitiveTypeInfo> columnTypes) {
+ // Timestamp column
+ columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+ columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+ // Dimension columns
+ for (DimensionSpec ds : query.getDimensions()) {
+ columnNames.add(ds.getOutputName());
+ columnTypes.add(TypeInfoFactory.stringTypeInfo);
+ }
+ // Metric columns
+ for (String metric : query.getMetrics()) {
+ columnNames.add(metric);
+ columnTypes.add(TypeInfoFactory.floatTypeInfo);
+ }
+ }
+
+ /* GroupBy query */
+ private void inferSchema(GroupByQuery query, List<String> columnNames, List<PrimitiveTypeInfo> columnTypes) {
+ // Timestamp column
+ columnNames.add(DruidTable.DEFAULT_TIMESTAMP_COLUMN);
+ columnTypes.add(TypeInfoFactory.timestampTypeInfo);
+ // Dimension columns
+ for (DimensionSpec ds : query.getDimensions()) {
+ columnNames.add(ds.getOutputName());
+ columnTypes.add(TypeInfoFactory.stringTypeInfo);
+ }
+ // Aggregator columns
+ for (AggregatorFactory af : query.getAggregatorSpecs()) {
+ columnNames.add(af.getName());
+ columnTypes.add(DruidSerDeUtils.convertDruidToHiveType(af.getTypeName()));
+ }
+ // Post-aggregator columns
+ for (PostAggregator pa : query.getPostAggregatorSpecs()) {
+ columnNames.add(pa.getName());
+ columnTypes.add(TypeInfoFactory.floatTypeInfo);
+ }
+ }
+
+ @Override
+ public Class<? extends Writable> getSerializedClass() {
+ return NullWritable.class;
+ }
+
+ @Override
+ public Writable serialize(Object o, ObjectInspector objectInspector) throws SerDeException {
+ return NullWritable.get();
+ }
+
+ @Override
+ public SerDeStats getSerDeStats() {
+ throw new UnsupportedOperationException("SerdeStats not supported.");
+ }
+
+ @Override
+ public Object deserialize(Writable writable) throws SerDeException {
+ DruidWritable input = (DruidWritable) writable;
+ List<Object> output = Lists.newArrayListWithExpectedSize(columns.length);
+ for (int i = 0; i < columns.length; i++) {
+ final Object value = input.getValue().get(columns[i]);
+ if (value == null) {
+ output.add(null);
+ continue;
+ }
+ switch (types[i].getPrimitiveCategory()) {
+ case TIMESTAMP:
+ output.add(new TimestampWritable(new Timestamp((Long)value)));
+ break;
+ case LONG:
+ output.add(new LongWritable(((Number)value).longValue()));
+ break;
+ case FLOAT:
+ output.add(new FloatWritable(((Number)value).floatValue()));
+ break;
+ case STRING:
+ output.add(new Text(value.toString()));
+ break;
+ default:
+ throw new SerDeException("Unknown type: " + types[i].getPrimitiveCategory());
+ }
+ }
+ return output;
+ }
+
+ @Override
+ public ObjectInspector getObjectInspector() throws SerDeException {
+ return inspector;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
new file mode 100644
index 0000000..29b8845
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidSerDeUtils.java
@@ -0,0 +1,83 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utils class for Druid SerDe.
+ */
+public final class DruidSerDeUtils {
+
+ private static final Logger LOG = LoggerFactory.getLogger(DruidSerDeUtils.class);
+
+ protected static final String FLOAT_TYPE = "FLOAT";
+ protected static final String LONG_TYPE = "LONG";
+ protected static final String STRING_TYPE = "STRING";
+
+ /* This method converts from the String representation of Druid type
+ * to the corresponding Hive type */
+ public static PrimitiveTypeInfo convertDruidToHiveType(String typeName) {
+ typeName = typeName.toUpperCase();
+ switch(typeName) {
+ case FLOAT_TYPE:
+ return TypeInfoFactory.floatTypeInfo;
+ case LONG_TYPE:
+ return TypeInfoFactory.longTypeInfo;
+ case STRING_TYPE:
+ return TypeInfoFactory.stringTypeInfo;
+ default:
+ // This is a guard for special Druid types e.g. hyperUnique
+ // (http://druid.io/docs/0.9.1.1/querying/aggregations.html#hyperunique-aggregator).
+ // Currently, we do not support doing anything special with them in Hive.
+ // However, those columns are there, and they can be actually read as normal
+ // dimensions e.g. with a select query. Thus, we print the warning and just read them
+ // as String.
+ LOG.warn("Transformation to STRING for unknown type " + typeName);
+ return TypeInfoFactory.stringTypeInfo;
+ }
+ }
+
+ /* This method converts from the String representation of Druid type
+ * to the String representation of the corresponding Hive type */
+ public static String convertDruidToHiveTypeString(String typeName) {
+ typeName = typeName.toUpperCase();
+ switch(typeName) {
+ case FLOAT_TYPE:
+ return serdeConstants.FLOAT_TYPE_NAME;
+ case LONG_TYPE:
+ return serdeConstants.BIGINT_TYPE_NAME;
+ case STRING_TYPE:
+ return serdeConstants.STRING_TYPE_NAME;
+ default:
+ // This is a guard for special Druid types e.g. hyperUnique
+ // (http://druid.io/docs/0.9.1.1/querying/aggregations.html#hyperunique-aggregator).
+ // Currently, we do not support doing anything special with them in Hive.
+ // However, those columns are there, and they can be actually read as normal
+ // dimensions e.g. with a select query. Thus, we print the warning and just read them
+ // as String.
+ LOG.warn("Transformation to STRING for unknown type " + typeName);
+ return serdeConstants.STRING_TYPE_NAME;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/hive/blob/58d1befa/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
----------------------------------------------------------------------
diff --git a/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
new file mode 100644
index 0000000..812ae03
--- /dev/null
+++ b/druid-handler/src/java/org/apache/hadoop/hive/druid/serde/DruidTimeseriesQueryRecordReader.java
@@ -0,0 +1,93 @@
+/**
+ * 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.hadoop.hive.druid.serde;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.List;
+
+import org.apache.hadoop.hive.druid.DruidStorageHandlerUtils;
+import org.apache.hadoop.hive.ql.optimizer.calcite.druid.DruidTable;
+import org.apache.hadoop.io.NullWritable;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+
+import io.druid.query.Result;
+import io.druid.query.timeseries.TimeseriesQuery;
+import io.druid.query.timeseries.TimeseriesResultValue;
+
+/**
+ * Record reader for results for Druid TimeseriesQuery.
+ */
+public class DruidTimeseriesQueryRecordReader
+ extends DruidQueryRecordReader<TimeseriesQuery, Result<TimeseriesResultValue>> {
+
+ private Result<TimeseriesResultValue> current;
+
+ @Override
+ protected TimeseriesQuery createQuery(String content) throws IOException {
+ return DruidStorageHandlerUtils.JSON_MAPPER.readValue(content, TimeseriesQuery.class);
+ }
+
+ @Override
+ protected List<Result<TimeseriesResultValue>> createResultsList(InputStream content) throws IOException {
+ return DruidStorageHandlerUtils.SMILE_MAPPER.readValue(content,
+ new TypeReference<List<Result<TimeseriesResultValue>>>(){});
+ }
+
+ @Override
+ public boolean nextKeyValue() {
+ if (results.hasNext()) {
+ current = results.next();
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public NullWritable getCurrentKey() throws IOException, InterruptedException {
+ return NullWritable.get();
+ }
+
+ @Override
+ public DruidWritable getCurrentValue() throws IOException, InterruptedException {
+ // Create new value
+ DruidWritable value = new DruidWritable();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ value.getValue().putAll(current.getValue().getBaseObject());
+ return value;
+ }
+
+ @Override
+ public boolean next(NullWritable key, DruidWritable value) {
+ if (nextKeyValue()) {
+ // Update value
+ value.getValue().clear();
+ value.getValue().put(DruidTable.DEFAULT_TIMESTAMP_COLUMN, current.getTimestamp().getMillis());
+ value.getValue().putAll(current.getValue().getBaseObject());
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public float getProgress() throws IOException {
+ return results.hasNext() ? 0 : 1;
+ }
+
+}