[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324933169
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/reduce/BrokerReduceService.java
 ##
 @@ -377,19 +386,50 @@ private void setAggregationResults(@Nonnull 
BrokerResponseNative brokerResponseN
   }
 }
 
-// Extract final results and set them into the broker response.
-List reducedAggregationResults = new 
ArrayList<>(numAggregationFunctions);
-for (int i = 0; i < numAggregationFunctions; i++) {
-  Serializable resultValue = AggregationFunctionUtils
-  
.getSerializableValue(aggregationFunctions[i].extractFinalResult(intermediateResults[i]));
+// The DISTINCT query is just another SELECTION style query from the 
user's point of view
+// and will return one or records in the result table for the column 
selected.
+// Internally the execution is happening as an aggregation function (but 
that is an implementation
+// detail) and so for that reason, response from broker should be a 
selection query result
+// up until now, we have treated DISTINCT similar to another aggregation 
function even in terms
+// of the result from function since it has been implemented as an 
aggregation function.
+// However, the broker response will be a selection query response as that 
makes sense from SQL
+// perspective
+if (isDistinct(aggregationFunctions)) {
+  final Object merged = intermediateResults[0];
 
 Review comment:
   (nit) final


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324929930
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/OutputColumnListAstNode.java
 ##
 @@ -41,8 +43,90 @@ public void addChild(AstNode childNode) {
 }
   }
 
+  /**
+   * Check for following style PQLs and raise syntax error
+   *
+   * These 4 queries are not valid SQL queries as well so PQL won't support 
them too
+   * (1) SELECT sum(col1), min(col2), DISTINCT(col3, col4)
+   * (2) SELECT col1, col2, DISTINCT(col3) FROM foo
+   * (3) SELECT DISTINCT(col1, col2), DISTINCT(col3) FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), 
DISTINCT(DaysSinceEpoch) FROM foo
+   *
+   * These 3 queries are either both selection and aggregation query
+   * or the query does not make sense from result point of view (like 6)
+   * (5) SELECT DISTINCT(col1), col2, col3 FROM foo
+   * (6) SELECT DISTINCT(col1), sum(col3), min(col4) FROM foo
+   * (7) SELECT DISTINCT(DaysSinceEpoch), 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   *
+   * SQL versions of the above queries:
+   *
+   * (1) SELECT sum(col1), min(col2), DISTINCT col3, col4
+   * (2) SELECT col1, col2, DISTINCT col3 FROM foo
+   * (3) SELECT DISTINCT col1, col2, DISTINCT col3 FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), DISTINCT 
DaysSinceEpoch FROM foo
+   *
+   * 1, 2, 3 and 4 will still not be supported in compliance with SQL
+   *
+   * (5) SELECT DISTINCT col1, col2, col3 FROM foo
+   * will be supported as it effectively becomes a multi column distinct
+   *
+   * (6) SELECT DISTINCT col1, sum(col3), min(col4) FROM foo
+   * although a valid SQL syntax for multi column distinct, if we decide to 
support
+   * them, we will have to do sum and min as transform functions which is not 
the case today.
+   * In any case, not a helpful query.
+   *
+   * (7) SELECT DISTINCT DaysSinceEpoch, 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   * again a valid SQL syntax for multi column distinct, we can support this 
since timeConvert
+   * is a valid supported transform function.
+   */
+  private void validate() {
+boolean identifierPresent = false;
+boolean distinctPresent = false;
+boolean functionPresent = false;
+if (hasChildren()) {
+  for (AstNode child : getChildren()) {
+if (child instanceof OutputColumnAstNode) {
+  if (child.hasChildren()) {
 
 Review comment:
   I think this if check is redundant?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324932815
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,169 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private final int _limit;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_limit = limit;
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder 
aggregationResultHolder,
+  @Nonnull BlockValSet... blockValSets) {
+Preconditions.checkArgument(blockValSets.length == columnNames.length, 
"Error invalid number of block value sets");
+
+if (_dataTypes == null) {
+  _dataTypes = new FieldSpec.DataType[columnNames.length];
+  for (int i = 0; i < blockValSets.length; i++) {
+_dataTypes[i] = blockValSets[i].getValueType();
+  }
+  _distinctTable.setColumnNames(columnNames);
+  _distinctTable.setColumnTypes(_dataTypes);
+}
+
+// TODO: Follow up PR will make few changes to start using 
DictionaryBasedAggregationOperator
+// for DISTINCT queries without filter.
+TransformBlockDataFetcher transformBlockDataFetcher =
+new TransformBlockDataFetcher(blockValSets, new Dictionary[0], new 
TransformResultMetadata[0]);
+
+int rowIndex = 0;
+// TODO: Do early termination in the operator itself which should
+// not call aggregate function at all if the limit has reached
+// that will require the interface change since this function
+// has to communicate back that required number of records have
+// been collected
+while (rowIndex < length && _distinctTable.size() < _limit) {
+  Object[] columnData = transformBlockDataFetcher.getRow(rowIndex);
+  _distinctTable.addKey(new Key(columnData));
+  ++rowIndex;
 
 Review comment:
   (nit) `rowIndex++`


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324929054
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,66 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+StringBuilder sb = new StringBuilder();
+int numOperands = operands.size();
+for (int i = 0; i < numOperands; ++i) {
+  Expression expression = operands.get(i);
 
 Review comment:
   Deduplicate the columns


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324928736
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,66 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
 
 Review comment:
   Do we want to support `SELECT DISTINCT * FROM table`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324926810
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,66 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+StringBuilder sb = new StringBuilder();
+int numOperands = operands.size();
+for (int i = 0; i < numOperands; ++i) {
 
 Review comment:
   (nit) `i++` for consistency


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324926552
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,66 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
 
 Review comment:
   (nit) `if (operands == null || operands.isEmpty())`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324930760
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
 ##
 @@ -174,13 +185,24 @@ private static String removeOptionsFromSql(String sql) {
 return matcher.replaceAll("");
   }
 
+  private static List convertDistinctSelectList(SqlNodeList 
selectList) {
+List selectExpr = new ArrayList<>();
+if (!Pql2Compiler.ENABLE_DISTINCT) {
+  throw new SqlCompilationException("Support for DISTINCT is currently 
disabled in Pinot");
+}
+
selectExpr.add(convertDistinctAndSelectListToFunctionExpression(selectList));
+return selectExpr;
+  }
+
   private static List convertSelectList(SqlNodeList selectList) {
 List selectExpr = new ArrayList<>();
+
 final Iterator iterator = selectList.iterator();
 while (iterator.hasNext()) {
   final SqlNode next = iterator.next();
   selectExpr.add(toExpression(next));
 
 Review comment:
   Not part of this PR, but we should deduplicate the columns


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324926410
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -111,6 +113,7 @@ private void convertGroupBy(PinotQuery pinotQuery, 
BrokerRequest brokerRequest)
   }
 
   private void convertSelectList(PinotQuery pinotQuery, BrokerRequest 
brokerRequest) {
+
 
 Review comment:
   (nit) revert this new line


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324931411
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -277,6 +281,36 @@ public HyperLogLog deserialize(ByteBuffer byteBuffer) {
 }
   };
 
+  public static final ObjectSerDe DISTINCT_TABLE_SER_DE = new 
ObjectSerDe() {
+
+@Override
+public byte[] serialize(DistinctTable distinctTable) {
+  try {
+return distinctTable.toBytes();
+  } catch (IOException e) {
+throw new IllegalStateException("Caught exception while serializing 
DistinctHolder", e);
 
 Review comment:
   "Caught exception while serializing Distinct**Table**"


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324931933
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DefaultAggregationExecutor.java
 ##
 @@ -56,12 +77,25 @@ public DefaultAggregationExecutor(@Nonnull 
AggregationFunctionContext[] function
   public void aggregate(@Nonnull TransformBlock transformBlock) {
 int length = transformBlock.getNumDocs();
 for (int i = 0; i < _numFunctions; i++) {
-  AggregationFunction function = _functions[i];
-  AggregationResultHolder resultHolder = _resultHolders[i];
-
+  final AggregationFunction function = _functions[i];
 
 Review comment:
   (nit) revert the **final**s


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324929120
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/FunctionCallAstNode.java
 ##
 @@ -75,22 +80,53 @@ public String getExpression() {
 return _expression;
   }
 
-  public AggregationInfo buildAggregationInfo() {
+  AggregationInfo buildAggregationInfo() {
 String expression;
 // COUNT aggregation function always works on '*'
-if (_name.equalsIgnoreCase("count")) {
+if (_name.equalsIgnoreCase(AggregationFunctionType.COUNT.getName())) {
   expression = "*";
 } else {
   List children = getChildren();
-  if (children == null || children.size() != 1) {
-throw new Pql2CompilationException("Aggregation function expects exact 
1 argument");
+  if (children == null || children.size() < 1) {
+throw new Pql2CompilationException("Aggregation function expects 
non-null argument");
+  }
+  if (!_name.equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) 
{
+if (children.size() != 1) {
+  throw new Pql2CompilationException("Aggregation function expects 
exactly 1 argument as column name");
+} else {
+  expression = 
TransformExpressionTree.getStandardExpression(children.get(0));
+}
+  } else {
+// DISTINCT
+if (!Pql2Compiler.ENABLE_DISTINCT) {
+  throw new Pql2CompilationException("Support for DISTINCT is 
currently disabled");
+}
+if (children.size() == 1) {
+  // single column DISTINCT query
+  // e.g SELECT DISTINCT(col) FROM foo
+  expression = 
TransformExpressionTree.getStandardExpression(children.get(0));
+} else {
+  // multi-column DISTINCT query
+  // e.g SELECT DISTINCT(col1, col2) FROM foo
+  // we will pass down the column expression to execution code
+  // as col1:col2
+  StringBuilder distinctColumnExpr = new StringBuilder();
+  int numChildren = children.size();
+  for (int i = 0; i < numChildren; ++i) {
+expression = 
TransformExpressionTree.getStandardExpression(children.get(i));
 
 Review comment:
   Deduplicate the columns


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324932329
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,169 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private final int _limit;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_limit = limit;
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int length, AggregationResultHolder 
aggregationResultHolder,
+  @Nonnull BlockValSet... blockValSets) {
+Preconditions.checkArgument(blockValSets.length == columnNames.length, 
"Error invalid number of block value sets");
+
+if (_dataTypes == null) {
+  _dataTypes = new FieldSpec.DataType[columnNames.length];
+  for (int i = 0; i < blockValSets.length; i++) {
+_dataTypes[i] = blockValSets[i].getValueType();
+  }
+  _distinctTable.setColumnNames(columnNames);
+  _distinctTable.setColumnTypes(_dataTypes);
+}
+
+// TODO: Follow up PR will make few changes to start using 
DictionaryBasedAggregationOperator
+// for DISTINCT queries without filter.
+TransformBlockDataFetcher transformBlockDataFetcher =
+new TransformBlockDataFetcher(blockValSets, new Dictionary[0], new 
TransformResultMetadata[0]);
+
+int rowIndex = 0;
+// TODO: Do early termination in the operator itself which should
+// not call aggregate function at all if the limit has reached
+// that will require the interface change since this function
+// has to communicate back that required number of records have
+// been collected
+while (rowIndex < length && _distinctTable.size() < _limit) {
+  Object[] columnData = transformBlockDataFetcher.getRow(rowIndex);
+  _distinctTable.addKey(new Key(columnData));
+  ++rowIndex;
+}
+  }
+
+  @Nonnull
+  @Override
+  public DistinctTable 

[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324932138
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,228 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final double LOAD_FACTOR = 0.75;
+  private static final int MAX_INITIAL_CAPACITY = 64 * 1024;
+  private FieldSpec.DataType[] _columnTypes;
+  private String[] _columnNames;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+_table.add(key);
+  }
+
+  public DistinctTable(int limit) {
+// TODO: see if 64k is the right max initial capacity to use
+// if it turns out that users always use LIMIT N > 0.75 * 64k and
+// there are indeed that many records, then there will be resizes.
+// The current method of setting the initial capacity as
+// min(64k, limit/loadFactor) will not require resizes for LIMIT N
+// where N <= 48000
+int initialCapacity = Math.min(MAX_INITIAL_CAPACITY, 
Math.abs(nextPowerOfTwo((int) (limit / LOAD_FACTOR;
+_table = new HashSet<>(initialCapacity);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(ByteBuffer byteBuffer)
+  throws IOException {
+DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
+DataSchema dataSchema = dataTable.getDataSchema();
+int numRows = dataTable.getNumberOfRows();
+int numColumns = dataSchema.size();
+
+_table = new HashSet<>();
+
+// extract rows from the datatable
+for (int rowIndex = 0; rowIndex < numRows; rowIndex++) {
+  Object[] columnValues = new Object[numColumns];
+  for (int colIndex = 0; colIndex < numColumns; colIndex++) {
+DataSchema.ColumnDataType columnDataType = 
dataSchema.getColumnDataType(colIndex);
+switch (columnDataType) {
+  case INT:
+columnValues[colIndex] = dataTable.getInt(rowIndex, colIndex);
+break;
+  case LONG:
+columnValues[colIndex] = dataTable.getLong(rowIndex, colIndex);
+break;
+  case FLOAT:
+columnValues[colIndex] = dataTable.getFloat(rowIndex, colIndex);
+break;
+  case DOUBLE:
+columnValues[colIndex] = dataTable.getDouble(rowIndex, colIndex);
+break;
+  case STRING:
+columnValues[colIndex] = dataTable.getString(rowIndex, colIndex);
+break;
+  default:
 
 Review comment:
   BYTES support?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324928495
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/FunctionCallAstNode.java
 ##
 @@ -75,22 +80,53 @@ public String getExpression() {
 return _expression;
   }
 
-  public AggregationInfo buildAggregationInfo() {
+  AggregationInfo buildAggregationInfo() {
 String expression;
 // COUNT aggregation function always works on '*'
-if (_name.equalsIgnoreCase("count")) {
+if (_name.equalsIgnoreCase(AggregationFunctionType.COUNT.getName())) {
   expression = "*";
 } else {
   List children = getChildren();
-  if (children == null || children.size() != 1) {
-throw new Pql2CompilationException("Aggregation function expects exact 
1 argument");
+  if (children == null || children.size() < 1) {
 
 Review comment:
   (nit) if (children == null || children.isEmpty())


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324929754
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/OutputColumnListAstNode.java
 ##
 @@ -41,8 +43,87 @@ public void addChild(AstNode childNode) {
 }
   }
 
+  /**
+   * Check for following style PQLs and raise syntax error
+   *
+   * These 4 queries are not valid SQL queries as well so PQL won't support 
them too
+   * (1) SELECT sum(col1), min(col2), DISTINCT(col3, col4)
+   * (2) SELECT col1, col2, DISTINCT(col3) FROM foo
+   * (3) SELECT DISTINCT(col1, col2), DISTINCT(col3) FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), 
DISTINCT(DaysSinceEpoch) FROM foo
+   *
+   * These 3 queries are either both selection and aggregation query
+   * or the query does not make sense from result point of view (like 6)
+   * (5) SELECT DISTINCT(col1), col2, col3 FROM foo
+   * (6) SELECT DISTINCT(col1), sum(col3), min(col4) FROM foo
+   * (7) SELECT DISTINCT(DaysSinceEpoch), 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   *
+   * SQL versions of the above queries:
+   *
+   * (1) SELECT sum(col1), min(col2), DISTINCT col3, col4
+   * (2) SELECT col1, col2, DISTINCT col3 FROM foo
+   * (3) SELECT DISTINCT col1, col2, DISTINCT col3 FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), DISTINCT 
DaysSinceEpoch FROM foo
+   *
+   * 1, 2, 3 and 4 will still not be supported in compliance with SQL
+   *
+   * (5) SELECT DISTINCT col1, col2, col3 FROM foo
+   * will be supported as it effectively becomes a multi column distinct
+   *
+   * (6) SELECT DISTINCT col1, sum(col3), min(col4) FROM foo
+   * although a valid SQL syntax for multi column distinct, if we decide to 
support
+   * them, we will have to do sum and min as transform functions which is not 
the case today.
+   * In any case, not a helpful query.
+   *
+   * (7) SELECT DISTINCT DaysSinceEpoch, 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   * again a valid SQL syntax for multi column distinct, we can support this 
since timeConvert
+   * is a valid supported transform function.
+   */
+  private void validate() {
+boolean identifierPresent = false;
+boolean distinctPresent = false;
+boolean functionPresent = false;
+if (hasChildren()) {
+  for (AstNode child : getChildren()) {
+if (child instanceof OutputColumnAstNode) {
 
 Review comment:
   Can you add comments on that? Also, if StarAstNode exists, there should be 
no other nodes based on the sql standard. We should have check on that


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-09-16 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r324929193
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/FunctionCallAstNode.java
 ##
 @@ -75,22 +80,53 @@ public String getExpression() {
 return _expression;
   }
 
-  public AggregationInfo buildAggregationInfo() {
+  AggregationInfo buildAggregationInfo() {
 String expression;
 // COUNT aggregation function always works on '*'
-if (_name.equalsIgnoreCase("count")) {
+if (_name.equalsIgnoreCase(AggregationFunctionType.COUNT.getName())) {
   expression = "*";
 } else {
   List children = getChildren();
-  if (children == null || children.size() != 1) {
-throw new Pql2CompilationException("Aggregation function expects exact 
1 argument");
+  if (children == null || children.size() < 1) {
+throw new Pql2CompilationException("Aggregation function expects 
non-null argument");
+  }
+  if (!_name.equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName())) 
{
+if (children.size() != 1) {
+  throw new Pql2CompilationException("Aggregation function expects 
exactly 1 argument as column name");
+} else {
+  expression = 
TransformExpressionTree.getStandardExpression(children.get(0));
+}
+  } else {
+// DISTINCT
+if (!Pql2Compiler.ENABLE_DISTINCT) {
+  throw new Pql2CompilationException("Support for DISTINCT is 
currently disabled");
+}
+if (children.size() == 1) {
+  // single column DISTINCT query
+  // e.g SELECT DISTINCT(col) FROM foo
+  expression = 
TransformExpressionTree.getStandardExpression(children.get(0));
 
 Review comment:
   Do we want to support `SELECT DISTINCT * FROM table`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706623
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,165 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private boolean init = false;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_dataTypes = new FieldSpec.DataType[columnNames.length];
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int records, AggregationResultHolder 
aggregationResultHolder,
+  @Nonnull BlockValSet... blockValSets) {
+Preconditions.checkArgument(blockValSets.length == columnNames.length, 
"Error invalid number of block value sets");
+
+if (!init) {
+  for (int i = 0; i < blockValSets.length; i++) {
+_dataTypes[i] = blockValSets[i].getValueType();
+  }
+  _distinctTable.setProjectedColumnNames(columnNames);
+  _distinctTable.setProjectedColumnTypes(_dataTypes);
+  init = true;
+}
+
+// TODO: Follow up PR will make few changes to start using 
DictionaryBasedAggregationOperator
+// for DISTINCT queries without filter.
+TransformBlockDataFetcher transformBlockDataFetcher = new 
TransformBlockDataFetcher(blockValSets, new Dictionary[0], new 
TransformResultMetadata[0]);
+
+int rowIndex = 0;
+while (rowIndex < records) {
+  Object[] columnData = transformBlockDataFetcher.getRow(rowIndex);
+  _distinctTable.addKey(new Key(columnData));
+  ++rowIndex;
+}
+  }
+
+  @Nonnull
+  @Override
+  public DistinctTable extractAggregationResult(AggregationResultHolder 
aggregationResultHolder) {
+return _distinctTable;
+  }
+
+  @Nonnull
+  @Override
+  public DistinctTable merge(DistinctTable inProgressMergedResult, 
DistinctTable newResultToMerge) {
+// do the union
+final Iterator iterator = newResultToMerge.getIterator();
 
 Review 

[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705938
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
 
 Review comment:
   (Major) Why do we create map with such big initial capacity? It does not 
make sense for small limit right? We can make a cap on the initial capacity, 
and use `Math.min(MAX_INITIAL_CAPACITY, limit / loadFactor)`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703392
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,65 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(FunctionCallAstNode.DISTINCT_FUNCTION_NAME)) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+final StringBuilder sb = new StringBuilder();
 
 Review comment:
   (nit) no final keyword :-P


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705588
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/plan/TransformPlanNode.java
 ##
 @@ -63,7 +65,8 @@ public TransformPlanNode(@Nonnull IndexSegment indexSegment, 
@Nonnull BrokerRequ
 _segmentName = indexSegment.getSegmentName();
 extractColumnsAndTransforms(brokerRequest, indexSegment);
 _projectionPlanNode =
-new ProjectionPlanNode(indexSegment, _projectionColumns, new 
DocIdSetPlanNode(indexSegment, brokerRequest, _maxDocPerNextCall));
+new ProjectionPlanNode(indexSegment, _projectionColumns,
 
 Review comment:
   (nit) Seems not formatted correctly


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705485
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/transform/TransformBlockDataFetcher.java
 ##
 @@ -19,11 +19,7 @@
 package org.apache.pinot.core.operator.transform;
 
 Review comment:
   Why do we need to touch this class? This is for selection queries only


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705237
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -425,8 +459,19 @@ public TDigest deserialize(ByteBuffer byteBuffer) {
   };
 
   // NOTE: DO NOT change the order, it has to be the same order as the 
ObjectType
-  private static final ObjectSerDe[] SER_DES =
-  {STRING_SER_DE, LONG_SER_DE, DOUBLE_SER_DE, DOUBLE_ARRAY_LIST_SER_DE, 
AVG_PAIR_SER_DE, MIN_MAX_RANGE_PAIR_SER_DE, HYPER_LOG_LOG_SER_DE, 
QUANTILE_DIGEST_SER_DE, MAP_SER_DE, INT_SET_SER_DE, TDIGEST_SER_DE};
+  private static final ObjectSerDe[] SER_DES = {
+  STRING_SER_DE,
+  LONG_SER_DE,
+  DOUBLE_SER_DE,
+  DOUBLE_ARRAY_LIST_SER_DE,
+  AVG_PAIR_SER_DE,
+  MIN_MAX_RANGE_PAIR_SER_DE,
+  HYPER_LOG_LOG_SER_DE,
+  QUANTILE_DIGEST_SER_DE,
+  MAP_SER_DE,
+  INT_SET_SER_DE,
+  TDIGEST_SER_DE,
+  DISTINCT_TABLE_SER_DE};
 
 Review comment:
   Back brace should be on a new line


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706434
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,165 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
 
 Review comment:
   No need for a separate init, just check `if (_dataTypes == null)`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705294
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/transform/TransformBlockDataFetcher.java
 ##
 @@ -49,7 +49,32 @@ public TransformBlockDataFetcher(BlockValSet[] 
blockValSets, Dictionary[] dictio
 return row;
   }
 
-  Fetcher createFetcher(BlockValSet blockValSet,
+  /**
+   * Create SV non-dictionary based fetcher
+   * @param blockValSet column value set
+   * @return fetcher
+   */
+  private Fetcher createSVNoDictionaryFetcher(BlockValSet blockValSet) {
+switch (blockValSet.getValueType()) {
+  case INT:
+return new SVIntValueFetcher(blockValSet.getIntValuesSV());
+  case LONG:
+return new SVLongValueFetcher(blockValSet.getLongValuesSV());
+  case FLOAT:
+return new SVFloatValueFetcher(blockValSet.getFloatValuesSV());
+  case DOUBLE:
+return new SVDoubleValueFetcher(blockValSet.getDoubleValuesSV());
+  case BOOLEAN:
 
 Review comment:
   BOOLEAN should not appear in this level. Internally we don't have BOOLEAN 
type


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706043
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
+  return;
+}
+
+_table.add(key);
+  }
+
+  public DistinctTable(int recordLimit) {
+_recordLimit = recordLimit;
+_table = new HashSet<>(INITIAL_CAPACITY);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(final ByteBuffer byteBuffer) throws IOException {
+final DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
+final DataSchema dataSchema = dataTable.getDataSchema();
+final int numRows = dataTable.getNumberOfRows();
+final int numColumns = dataSchema.size();
+
+_table = new HashSet<>();
+
+// extract rows from the datatable
+for (int rowIndex  = 0; rowIndex < numRows; rowIndex++) {
+ Object[] columnValues = new Object[numColumns];
+  for (int colIndex = 0; colIndex < numColumns; colIndex++) {
+DataSchema.ColumnDataType columnDataType = 
dataSchema.getColumnDataType(colIndex);
+switch (columnDataType) {
+  case INT:
+columnValues[colIndex] = dataTable.getInt(rowIndex, colIndex);
+break;
+  case LONG:
+columnValues[colIndex] = dataTable.getLong(rowIndex, colIndex);
+break;
+  case STRING:
 
 Review comment:
   (nit) put STRING after DOUBLE


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703705
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/FunctionCallAstNode.java
 ##
 @@ -33,6 +36,11 @@
   private String _expression;
   private boolean _isInSelectList;
 
+  public static final String DISTINCT_MULTI_COLUMN_SEPARATOR = ":";
+  public static final String COLUMN_KEY_IN_AGGREGATION_INFO = "column";
+  public static final String COUNT_FUNCTION_NAME = "count";
 
 Review comment:
   Use AggregationFunctionType.COUNT.getName(), same for DISTINCT


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703463
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,65 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(FunctionCallAstNode.DISTINCT_FUNCTION_NAME)) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+final StringBuilder sb = new StringBuilder();
+for (int i = 0; i < operands.size() ; ++i) {
 
 Review comment:
   Cache operands.size()


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705994
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
+  return;
+}
+
+_table.add(key);
+  }
+
+  public DistinctTable(int recordLimit) {
+_recordLimit = recordLimit;
+_table = new HashSet<>(INITIAL_CAPACITY);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(final ByteBuffer byteBuffer) throws IOException {
+final DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
 
 Review comment:
   (nit) final


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319704774
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
 ##
 @@ -213,6 +231,32 @@ private static Expression convertOrderBy(SqlNode node) {
 return expression;
   }
 
+  /**
+   * DISTINCT is implemented as an aggregation function so need to take the 
select list items
+   * and convert them into a single function expression for handing over to 
execution engine
+   * either as a PinotQuery or BrokerRequest via conversion
+   * @param selectList select list items
+   * @return DISTINCT function expression
+   */
+  private static Expression 
convertDistinctAndSelectListToFunctionExpression(final SqlNodeList selectList) {
+final String functionName = FunctionCallAstNode.DISTINCT_FUNCTION_NAME;
 
 Review comment:
   (nit) final keyword and `AggregationFunctionType.DISTINCT.name()`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706073
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
 
 Review comment:
   Let's just name these two variables _columnTypes and _columnNames


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705306
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/operator/transform/TransformBlockDataFetcher.java
 ##
 @@ -49,7 +49,32 @@ public TransformBlockDataFetcher(BlockValSet[] 
blockValSets, Dictionary[] dictio
 return row;
   }
 
-  Fetcher createFetcher(BlockValSet blockValSet,
+  /**
+   * Create SV non-dictionary based fetcher
+   * @param blockValSet column value set
+   * @return fetcher
+   */
+  private Fetcher createSVNoDictionaryFetcher(BlockValSet blockValSet) {
+switch (blockValSet.getValueType()) {
+  case INT:
+return new SVIntValueFetcher(blockValSet.getIntValuesSV());
+  case LONG:
+return new SVLongValueFetcher(blockValSet.getLongValuesSV());
+  case FLOAT:
+return new SVFloatValueFetcher(blockValSet.getFloatValuesSV());
+  case DOUBLE:
+return new SVDoubleValueFetcher(blockValSet.getDoubleValuesSV());
+  case BOOLEAN:
+  case STRING:
+return new SVStringValueFetcher(blockValSet.getStringValuesSV());
+  case BYTES:
+return new SVBytesValueFetcher(blockValSet.getBytesValuesSV());
+}
+
+throw new UnsupportedOperationException();
 
 Review comment:
   Move this to `default` and throw `IllegalStateException`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706007
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
+  return;
+}
+
+_table.add(key);
+  }
+
+  public DistinctTable(int recordLimit) {
+_recordLimit = recordLimit;
+_table = new HashSet<>(INITIAL_CAPACITY);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(final ByteBuffer byteBuffer) throws IOException {
+final DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
+final DataSchema dataSchema = dataTable.getDataSchema();
+final int numRows = dataTable.getNumberOfRows();
+final int numColumns = dataSchema.size();
+
+_table = new HashSet<>();
+
+// extract rows from the datatable
+for (int rowIndex  = 0; rowIndex < numRows; rowIndex++) {
 
 Review comment:
   format the class


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705177
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -277,6 +281,36 @@ public HyperLogLog deserialize(ByteBuffer byteBuffer) {
 }
   };
 
+  public static final ObjectSerDe DISTINCT_TABLE_SER_DE = new 
ObjectSerDe() {
+
+@Override
+public byte[] serialize(DistinctTable tupleHolder) {
+  try {
+return tupleHolder.toBytes();
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while serializing 
TupleHolder", e);
 
 Review comment:
   TupleHolder? Also let's throw IllegalStateException


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706124
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
+  return;
+}
+
+_table.add(key);
+  }
+
+  public DistinctTable(int recordLimit) {
+_recordLimit = recordLimit;
+_table = new HashSet<>(INITIAL_CAPACITY);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(final ByteBuffer byteBuffer) throws IOException {
+final DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
+final DataSchema dataSchema = dataTable.getDataSchema();
+final int numRows = dataTable.getNumberOfRows();
+final int numColumns = dataSchema.size();
+
+_table = new HashSet<>();
+
+// extract rows from the datatable
+for (int rowIndex  = 0; rowIndex < numRows; rowIndex++) {
+ Object[] columnValues = new Object[numColumns];
+  for (int colIndex = 0; colIndex < numColumns; colIndex++) {
+DataSchema.ColumnDataType columnDataType = 
dataSchema.getColumnDataType(colIndex);
+switch (columnDataType) {
+  case INT:
+columnValues[colIndex] = dataTable.getInt(rowIndex, colIndex);
+break;
+  case LONG:
+columnValues[colIndex] = dataTable.getLong(rowIndex, colIndex);
+break;
+  case STRING:
+columnValues[colIndex] = dataTable.getString(rowIndex, colIndex);
+break;
+  case FLOAT:
+columnValues[colIndex] = dataTable.getFloat(rowIndex, colIndex);
+break;
+  case DOUBLE:
+columnValues[colIndex] = dataTable.getDouble(rowIndex, colIndex);
+break;
+  default:
+throw new UnsupportedOperationException("Unexpected column data 
type " + columnDataType + " in data table for DISTINCT query");
 
 Review comment:
   (nit) Prefer IllegalStateException


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706548
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,165 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private boolean init = false;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_dataTypes = new FieldSpec.DataType[columnNames.length];
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int records, AggregationResultHolder 
aggregationResultHolder,
 
 Review comment:
   (nit) first parameter `length`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319704582
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/OutputColumnListAstNode.java
 ##
 @@ -41,8 +43,87 @@ public void addChild(AstNode childNode) {
 }
   }
 
+  /**
+   * Check for following style PQLs and raise syntax error
+   *
+   * These 4 queries are not valid SQL queries as well so PQL won't support 
them too
+   * (1) SELECT sum(col1), min(col2), DISTINCT(col3, col4)
+   * (2) SELECT col1, col2, DISTINCT(col3) FROM foo
+   * (3) SELECT DISTINCT(col1, col2), DISTINCT(col3) FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), 
DISTINCT(DaysSinceEpoch) FROM foo
+   *
+   * These 3 queries are either both selection and aggregation query
+   * or the query does not make sense from result point of view (like 6)
+   * (5) SELECT DISTINCT(col1), col2, col3 FROM foo
+   * (6) SELECT DISTINCT(col1), sum(col3), min(col4) FROM foo
+   * (7) SELECT DISTINCT(DaysSinceEpoch), 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   *
+   * SQL versions of the above queries:
+   *
+   * (1) SELECT sum(col1), min(col2), DISTINCT col3, col4
+   * (2) SELECT col1, col2, DISTINCT col3 FROM foo
+   * (3) SELECT DISTINCT col1, col2, DISTINCT col3 FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), DISTINCT 
DaysSinceEpoch FROM foo
+   *
+   * 1, 2, 3 and 4 will still not be supported in compliance with SQL
+   *
+   * (5) SELECT DISTINCT col1, col2, col3 FROM foo
+   * will be supported as it effectively becomes a multi column distinct
+   *
+   * (6) SELECT DISTINCT col1, sum(col3), min(col4) FROM foo
+   * although a valid SQL syntax for multi column distinct, if we decide to 
support
+   * them, we will have to do sum and min as transform functions which is not 
the case today.
+   * In any case, not a helpful query.
+   *
+   * (7) SELECT DISTINCT DaysSinceEpoch, 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   * again a valid SQL syntax for multi column distinct, we can support this 
since timeConvert
+   * is a valid supported transform function.
+   */
+  private void validate() {
+boolean identifierPresent = false;
+boolean distinctPresent = false;
+boolean functionPresent = false;
+if (hasChildren()) {
+  for (AstNode child : getChildren()) {
+if (child instanceof OutputColumnAstNode) {
+  if (child.hasChildren()) {
+for (AstNode selectChild : child.getChildren()) {
+  if (selectChild instanceof IdentifierAstNode) {
+if (distinctPresent) {
+  throw new Pql2CompilationException("Syntax error: SELECT 
list columns should be part of DISTINCT clause");
+} else {
+  identifierPresent = true;
+}
+  } else if (selectChild instanceof FunctionCallAstNode) {
+FunctionCallAstNode function = 
(FunctionCallAstNode)selectChild;
+if (function.getName().equalsIgnoreCase("distinct")) {
 
 Review comment:
   Use constant


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706302
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java
 ##
 @@ -59,24 +63,35 @@ public static AggregationFunctionColumnPair 
getFunctionColumnPair(@Nonnull Aggre
 return new AggregationFunctionColumnPair(functionType, 
getColumn(aggregationInfo));
   }
 
+  public static boolean isDistinct(AggregationFunctionContext[] 
functionContexts) {
+return functionContexts.length == 1 &&
+functionContexts[0].getAggregationFunction().getType() == 
AggregationFunctionType.DISTINCT;
+  }
+
   /**
* Creates an {@link AggregationFunctionContext} from the {@link 
AggregationInfo}.
*/
   @Nonnull
   public static AggregationFunctionContext 
getAggregationFunctionContext(@Nonnull AggregationInfo aggregationInfo) {
-String functionName = aggregationInfo.getAggregationType();
-AggregationFunction aggregationFunction = 
AggregationFunctionFactory.getAggregationFunction(functionName);
-return new AggregationFunctionContext(aggregationFunction, 
AggregationFunctionUtils.getColumn(aggregationInfo));
+   return getAggregationFunctionContext(aggregationInfo, new BrokerRequest());
 
 Review comment:
   Pass in `null`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705227
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -425,8 +459,20 @@ public TDigest deserialize(ByteBuffer byteBuffer) {
   };
 
   // NOTE: DO NOT change the order, it has to be the same order as the 
ObjectType
-  private static final ObjectSerDe[] SER_DES =
-  {STRING_SER_DE, LONG_SER_DE, DOUBLE_SER_DE, DOUBLE_ARRAY_LIST_SER_DE, 
AVG_PAIR_SER_DE, MIN_MAX_RANGE_PAIR_SER_DE, HYPER_LOG_LOG_SER_DE, 
QUANTILE_DIGEST_SER_DE, MAP_SER_DE, INT_SET_SER_DE, TDIGEST_SER_DE};
+  private static final ObjectSerDe[] SER_DES = {
 
 Review comment:
   Comment to turn formatter on and off


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706586
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,165 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private boolean init = false;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_dataTypes = new FieldSpec.DataType[columnNames.length];
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int records, AggregationResultHolder 
aggregationResultHolder,
+  @Nonnull BlockValSet... blockValSets) {
+Preconditions.checkArgument(blockValSets.length == columnNames.length, 
"Error invalid number of block value sets");
+
+if (!init) {
+  for (int i = 0; i < blockValSets.length; i++) {
+_dataTypes[i] = blockValSets[i].getValueType();
+  }
+  _distinctTable.setProjectedColumnNames(columnNames);
+  _distinctTable.setProjectedColumnTypes(_dataTypes);
+  init = true;
+}
+
+// TODO: Follow up PR will make few changes to start using 
DictionaryBasedAggregationOperator
+// for DISTINCT queries without filter.
+TransformBlockDataFetcher transformBlockDataFetcher = new 
TransformBlockDataFetcher(blockValSets, new Dictionary[0], new 
TransformResultMetadata[0]);
+
+int rowIndex = 0;
+while (rowIndex < records) {
 
 Review comment:
   (Major) track distinct table size and early terminate if size reached. No 
need to track size inside distinct table


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706115
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
+  return;
+}
+
+_table.add(key);
+  }
+
+  public DistinctTable(int recordLimit) {
+_recordLimit = recordLimit;
+_table = new HashSet<>(INITIAL_CAPACITY);
+  }
+
+  /**
+   * DESERIALIZE: Broker side
+   * @param byteBuffer data to deserialize
+   * @throws IOException
+   */
+  public DistinctTable(final ByteBuffer byteBuffer) throws IOException {
+final DataTable dataTable = DataTableFactory.getDataTable(byteBuffer);
+final DataSchema dataSchema = dataTable.getDataSchema();
+final int numRows = dataTable.getNumberOfRows();
+final int numColumns = dataSchema.size();
+
+_table = new HashSet<>();
+
+// extract rows from the datatable
+for (int rowIndex  = 0; rowIndex < numRows; rowIndex++) {
+ Object[] columnValues = new Object[numColumns];
+  for (int colIndex = 0; colIndex < numColumns; colIndex++) {
+DataSchema.ColumnDataType columnDataType = 
dataSchema.getColumnDataType(colIndex);
+switch (columnDataType) {
+  case INT:
+columnValues[colIndex] = dataTable.getInt(rowIndex, colIndex);
+break;
+  case LONG:
+columnValues[colIndex] = dataTable.getLong(rowIndex, colIndex);
+break;
+  case STRING:
+columnValues[colIndex] = dataTable.getString(rowIndex, colIndex);
+break;
+  case FLOAT:
+columnValues[colIndex] = dataTable.getFloat(rowIndex, colIndex);
+break;
+  case DOUBLE:
+columnValues[colIndex] = dataTable.getDouble(rowIndex, colIndex);
+break;
+  default:
+throw new UnsupportedOperationException("Unexpected column data 
type " + columnDataType + " in data table for DISTINCT query");
+}
+  }
+
+  _table.add(new Key(columnValues));
+}
+
+_projectedColumnNames = dataSchema.getColumnNames();
+_projectedColumnTypes = buildDataTypesFromColumnTypes(dataSchema);
+  }
+
+  private FieldSpec.DataType[] buildDataTypesFromColumnTypes(final DataSchema 
dataSchema) {
+final int numColumns = dataSchema.size();
+final FieldSpec.DataType[] columnTypes = new 
FieldSpec.DataType[numColumns];
+for (int colIndex = 0; colIndex < numColumns; colIndex++) {
+  DataSchema.ColumnDataType columnDataType = 
dataSchema.getColumnDataType(colIndex);
+  switch (columnDataType) {
+case INT:
+  columnTypes[colIndex] = 

[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705084
 
 

 ##
 File path: pinot-common/src/thrift/request.thrift
 ##
 @@ -155,6 +155,7 @@ struct BrokerRequest {
  16: optional HavingFilterQueryMap havingFilterSubQueryMap;
  17: optional query.PinotQuery pinotQuery;
  18: optional list orderBy;
+ 19: optional i32 limit = 0;
 
 Review comment:
   Is the default 0 correct here?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319704979
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/sql/parsers/CalciteSqlParser.java
 ##
 @@ -174,13 +182,23 @@ private static String removeOptionsFromSql(String sql) {
 return matcher.replaceAll("");
   }
 
-  private static List convertSelectList(SqlNodeList selectList) {
+  private static List convertSelectList(SqlNodeList selectList, 
SqlNode distinctNode) {
 
 Review comment:
   I'd prefer break this method into two methods for clarity: 
`convertSelectList()` and `convertDistinctSelectList()`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705139
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -93,6 +95,8 @@ public static ObjectType getObjectType(Object value) {
 return ObjectType.IntSet;
   } else if (value instanceof TDigest) {
 return ObjectType.TDigest;
+  } else if (value instanceof 
org.apache.pinot.core.query.aggregation.DistinctTable) {
 
 Review comment:
   Replace `org.apache.pinot.core.query.aggregation.DistinctTable` with 
`DistinctTable`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706329
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/AggregationFunctionUtils.java
 ##
 @@ -89,12 +104,16 @@ public static AggregationFunctionContext 
getAggregationFunctionContext(@Nonnull
   }
 
   @Nonnull
-  public static AggregationFunction[] getAggregationFunctions(@Nonnull 
List aggregationInfos) {
+  public static AggregationFunction[] getAggregationFunctions(BrokerRequest 
brokerRequest) {
+List aggregationInfos = 
brokerRequest.getAggregationsInfo();
 int numAggregationFunctions = aggregationInfos.size();
 AggregationFunction[] aggregationFunctions = new 
AggregationFunction[numAggregationFunctions];
 for (int i = 0; i < numAggregationFunctions; i++) {
+  AggregationInfo aggregationInfo = aggregationInfos.get(i);
 
 Review comment:
   Revert?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703822
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/FunctionCallAstNode.java
 ##
 @@ -75,22 +83,52 @@ public String getExpression() {
 return _expression;
   }
 
-  public AggregationInfo buildAggregationInfo() {
+  AggregationInfo buildAggregationInfo() {
 String expression;
 // COUNT aggregation function always works on '*'
-if (_name.equalsIgnoreCase("count")) {
+if (_name.equalsIgnoreCase(COUNT_FUNCTION_NAME)) {
   expression = "*";
 } else {
   List children = getChildren();
-  if (children == null || children.size() != 1) {
-throw new Pql2CompilationException("Aggregation function expects exact 
1 argument");
+  if (children == null || children.size() < 1) {
+throw new Pql2CompilationException("Aggregation function expects 
non-null argument");
+  }
+  if (!_name.equalsIgnoreCase(DISTINCT_FUNCTION_NAME)) {
+if (children.size() != 1) {
+  throw new Pql2CompilationException("Aggregation function expects 
exactly 1 argument as column name");
+} else {
+  expression = 
TransformExpressionTree.getStandardExpression(children.get(0));
+}
+  } else {
+// DISTINCT
+if (!Pql2Compiler.ENABLE_DISTINCT) {
+  throw new Pql2CompilationException("Support for DISTINCT is 
currently disabled");
+}
+if (children.size() == 1) {
 
 Review comment:
   Same comments as in `PinotQuery2BrokerRequestConverter`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705744
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DefaultAggregationExecutor.java
 ##
 @@ -41,13 +44,29 @@ public DefaultAggregationExecutor(@Nonnull 
AggregationFunctionContext[] function
 _numFunctions = functionContexts.length;
 _functions = new AggregationFunction[_numFunctions];
 _resultHolders = new AggregationResultHolder[_numFunctions];
-_expressions = new TransformExpressionTree[_numFunctions];
-for (int i = 0; i < _numFunctions; i++) {
-  AggregationFunction function = 
functionContexts[i].getAggregationFunction();
-  _functions[i] = function;
-  _resultHolders[i] = _functions[i].createAggregationResultHolder();
-  if (function.getType() != AggregationFunctionType.COUNT) {
-_expressions[i] = 
TransformExpressionTree.compileToExpressionTree(functionContexts[i].getColumn());
+if (AggregationFunctionUtils.isDistinct(functionContexts)) {
+  // handle distinct (col1, col2..) function
+  // unlike other aggregate functions, distinct can work on multiple 
columns
+  // so we need to build expression tree for each column
+  _functions[0] = functionContexts[0].getAggregationFunction();
+  _resultHolders[0] = _functions[0].createAggregationResultHolder();
+  final String multiColumnExpression = functionContexts[0].getColumn();
 
 Review comment:
   (nit) final


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703535
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,65 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(FunctionCallAstNode.DISTINCT_FUNCTION_NAME)) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+final StringBuilder sb = new StringBuilder();
+for (int i = 0; i < operands.size() ; ++i) {
+  Expression expression = operands.get(i);
+  sb.append(getColumnExpression(expression));
+  if (i <= operands.size() - 2) {
 
 Review comment:
   Move this one line up, and check `if (i != 0)`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705200
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -277,6 +281,36 @@ public HyperLogLog deserialize(ByteBuffer byteBuffer) {
 }
   };
 
+  public static final ObjectSerDe DISTINCT_TABLE_SER_DE = new 
ObjectSerDe() {
+
+@Override
+public byte[] serialize(DistinctTable tupleHolder) {
+  try {
+return tupleHolder.toBytes();
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while serializing 
TupleHolder", e);
+  }
+}
+
+@Override
+public DistinctTable deserialize(byte[] bytes) {
+  try {
+return new DistinctTable(ByteBuffer.wrap(bytes));
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while de-serializing 
HyperLogLog", e);
+  }
+}
+
+@Override
+public DistinctTable deserialize(ByteBuffer byteBuffer) {
+  try {
+return new DistinctTable(byteBuffer);
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while de-serializing 
HyperLogLog", e);
 
 Review comment:
   HyperLogLog


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705615
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/plan/TransformPlanNode.java
 ##
 @@ -76,7 +79,18 @@ private void extractColumnsAndTransforms(@Nonnull 
BrokerRequest brokerRequest,
   IndexSegment indexSegment) {
 if (brokerRequest.isSetAggregationsInfo()) {
   for (AggregationInfo aggregationInfo : 
brokerRequest.getAggregationsInfo()) {
-if 
(!aggregationInfo.getAggregationType().equalsIgnoreCase(AggregationFunctionType.COUNT.getName()))
 {
+if 
(aggregationInfo.getAggregationType().equalsIgnoreCase(AggregationFunctionType.DISTINCT.getName()))
 {
+  // handle DISTINCT (col1, col2 ...) as an aggregate function
+  final String multiColumnExpression = 
AggregationFunctionUtils.getColumn(aggregationInfo);
 
 Review comment:
   (nit) final


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319706514
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/function/DistinctAggregationFunction.java
 ##
 @@ -0,0 +1,165 @@
+/**
+ * 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.pinot.core.query.aggregation.function;
+
+import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.Iterator;
+import javax.annotation.Nonnull;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.function.AggregationFunctionType;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.core.common.BlockValSet;
+import org.apache.pinot.core.data.table.Key;
+import org.apache.pinot.core.operator.transform.TransformBlockDataFetcher;
+import org.apache.pinot.core.operator.transform.TransformResultMetadata;
+import org.apache.pinot.core.query.aggregation.AggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.DistinctTable;
+import org.apache.pinot.core.query.aggregation.ObjectAggregationResultHolder;
+import org.apache.pinot.core.query.aggregation.groupby.GroupByResultHolder;
+import org.apache.pinot.core.segment.index.readers.Dictionary;
+import org.apache.pinot.pql.parsers.pql2.ast.FunctionCallAstNode;
+
+
+/**
+ * DISTINCT clause in SQL is implemented as function in the
+ * execution engine of Pinot.
+ */
+public class DistinctAggregationFunction implements 
AggregationFunction {
+
+  private final DistinctTable _distinctTable;
+  private final String[] columnNames;
+  private FieldSpec.DataType[] _dataTypes;
+  private boolean init = false;
+
+  DistinctAggregationFunction(String multiColumnExpression, int limit) {
+_distinctTable = new DistinctTable(limit);
+columnNames = 
multiColumnExpression.split(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+_dataTypes = new FieldSpec.DataType[columnNames.length];
+  }
+
+  @Nonnull
+  @Override
+  public AggregationFunctionType getType() {
+return AggregationFunctionType.DISTINCT;
+  }
+
+  @Nonnull
+  @Override
+  public String getColumnName(@Nonnull String column) {
+return AggregationFunctionType.DISTINCT.getName() + "_" + column;
+  }
+
+  @Nonnull
+  @Override
+  public DataSchema.ColumnDataType getIntermediateResultColumnType() {
+return DataSchema.ColumnDataType.OBJECT;
+  }
+
+  @Override
+  public void accept(@Nonnull AggregationFunctionVisitorBase visitor) {
+visitor.visit(this);
+  }
+
+  @Nonnull
+  @Override
+  public AggregationResultHolder createAggregationResultHolder() {
+return new ObjectAggregationResultHolder();
+  }
+
+  @Override
+  public void aggregate(int records, AggregationResultHolder 
aggregationResultHolder,
+  @Nonnull BlockValSet... blockValSets) {
+Preconditions.checkArgument(blockValSets.length == columnNames.length, 
"Error invalid number of block value sets");
+
+if (!init) {
+  for (int i = 0; i < blockValSets.length; i++) {
+_dataTypes[i] = blockValSets[i].getValueType();
+  }
+  _distinctTable.setProjectedColumnNames(columnNames);
+  _distinctTable.setProjectedColumnTypes(_dataTypes);
+  init = true;
+}
+
+// TODO: Follow up PR will make few changes to start using 
DictionaryBasedAggregationOperator
 
 Review comment:
   Without filter and on single column


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705186
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/common/ObjectSerDeUtils.java
 ##
 @@ -277,6 +281,36 @@ public HyperLogLog deserialize(ByteBuffer byteBuffer) {
 }
   };
 
+  public static final ObjectSerDe DISTINCT_TABLE_SER_DE = new 
ObjectSerDe() {
+
+@Override
+public byte[] serialize(DistinctTable tupleHolder) {
+  try {
+return tupleHolder.toBytes();
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while serializing 
TupleHolder", e);
+  }
+}
+
+@Override
+public DistinctTable deserialize(byte[] bytes) {
+  try {
+return new DistinctTable(ByteBuffer.wrap(bytes));
+  } catch (IOException e) {
+throw new RuntimeException("Caught exception while de-serializing 
HyperLogLog", e);
 
 Review comment:
   HyperLogLog


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705954
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
+  private FieldSpec.DataType[] _projectedColumnTypes;
+  private String[] _projectedColumnNames;
+  private int _recordLimit;
+  private Set _table;
+
+  /**
+   * Add a row to hash table
+   * @param key multi-column key to add
+   */
+  public void addKey(final Key key) {
+if (_table.size() >= _recordLimit) {
+  LOGGER.info("Distinct table Reached allowed max cardinality of {}", 
_recordLimit);
 
 Review comment:
   (Major) This will flood the log


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703376
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/PinotQuery2BrokerRequestConverter.java
 ##
 @@ -223,38 +226,65 @@ private String standardizeFunction(Function 
functionCall) {
 
   private AggregationInfo buildAggregationInfo(Function function) {
 List operands = function.getOperands();
-if (operands == null || operands.size() != 1) {
-  throw new Pql2CompilationException(
-  "Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+if (operands == null || operands.size() < 1) {
+  throw new Pql2CompilationException("Aggregation function expects non 
null argument");
 }
-String functionName = function.getOperator();
+
 String columnName;
-if (functionName.equalsIgnoreCase("count")) {
-  columnName = "*";
+String functionName = function.getOperator();
+
+if 
(functionName.equalsIgnoreCase(FunctionCallAstNode.DISTINCT_FUNCTION_NAME)) {
+  // DISTINCT can support multiple arguments
+  if (operands.size() == 1) {
+// single column DISTINCT
+columnName = getColumnExpression(operands.get(0));
+  } else {
+// multi column DISTINCT
+final StringBuilder sb = new StringBuilder();
+for (int i = 0; i < operands.size() ; ++i) {
+  Expression expression = operands.get(i);
+  sb.append(getColumnExpression(expression));
+  if (i <= operands.size() - 2) {
+sb.append(FunctionCallAstNode.DISTINCT_MULTI_COLUMN_SEPARATOR);
+  }
+}
+columnName = sb.toString();
+  }
 } else {
-  Expression functionParam = operands.get(0);
+  // other aggregation functions support exactly one argument
+  if (operands.size() != 1) {
+throw new Pql2CompilationException(
+"Aggregation function" + function.getOperator() + " expects 1 
argument. found: " + operands);
+  }
 
-  switch (functionParam.getType()) {
-case LITERAL:
-  columnName = functionParam.getLiteral().getStringValue();
-  break;
-case IDENTIFIER:
-  columnName = functionParam.getIdentifier().getName();
-  break;
-case FUNCTION:
-  columnName = standardizeExpression(functionParam, false, true);
-  break;
-default:
-  throw new UnsupportedOperationException("Unrecognized 
functionParamType:" + functionParam.getType());
+  if 
(functionName.equalsIgnoreCase(FunctionCallAstNode.COUNT_FUNCTION_NAME)) {
+columnName = "*";
+  } else {
+Expression functionParam = operands.get(0);
+columnName = getColumnExpression(functionParam);
   }
 }
+
 AggregationInfo aggregationInfo = new AggregationInfo();
 aggregationInfo.setAggregationType(functionName);
 aggregationInfo.putToAggregationParams("column", columnName);
 
 Review comment:
   Replace "column" with constant


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319705818
 
 

 ##
 File path: 
pinot-core/src/main/java/org/apache/pinot/core/query/aggregation/DistinctTable.java
 ##
 @@ -0,0 +1,235 @@
+/**
+ * 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.pinot.core.query.aggregation;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import org.apache.pinot.common.data.FieldSpec;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.common.utils.DataTable;
+import org.apache.pinot.core.common.datatable.DataTableBuilder;
+import org.apache.pinot.core.common.datatable.DataTableFactory;
+import org.apache.pinot.core.data.table.Key;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This serves the following purposes:
+ *
+ * (1) Intermediate result object for Distinct aggregation function
+ * (2) The same object is serialized by the server inside the data table
+ * for sending the results to broker. Broker deserializes it.
+ */
+public class DistinctTable {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(DistinctTable.class);
+  private final static int INITIAL_CAPACITY = 64000;
 
 Review comment:
   (nit) private static final


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319703617
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/Pql2Compiler.java
 ##
 @@ -71,6 +71,9 @@
   Boolean.valueOf(System.getProperty("pinot.query.converter.validate", 
"false"));
   public static boolean FAIL_ON_CONVERSION_ERROR =
   
Boolean.valueOf(System.getProperty("pinot.query.converter.fail_on_error", 
"false"));
+  public static String ENABLE_DISTINCT_KEY = "pinot.distinct.enabled";
 
 Review comment:
   I don't think we need to disable this feature @mayankshriv 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org



[GitHub] [incubator-pinot] Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT clause

2019-08-30 Thread GitBox
Jackie-Jiang commented on a change in pull request #4535: Implement DISTINCT 
clause
URL: https://github.com/apache/incubator-pinot/pull/4535#discussion_r319704238
 
 

 ##
 File path: 
pinot-common/src/main/java/org/apache/pinot/pql/parsers/pql2/ast/OutputColumnListAstNode.java
 ##
 @@ -41,8 +43,87 @@ public void addChild(AstNode childNode) {
 }
   }
 
+  /**
+   * Check for following style PQLs and raise syntax error
+   *
+   * These 4 queries are not valid SQL queries as well so PQL won't support 
them too
+   * (1) SELECT sum(col1), min(col2), DISTINCT(col3, col4)
+   * (2) SELECT col1, col2, DISTINCT(col3) FROM foo
+   * (3) SELECT DISTINCT(col1, col2), DISTINCT(col3) FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), 
DISTINCT(DaysSinceEpoch) FROM foo
+   *
+   * These 3 queries are either both selection and aggregation query
+   * or the query does not make sense from result point of view (like 6)
+   * (5) SELECT DISTINCT(col1), col2, col3 FROM foo
+   * (6) SELECT DISTINCT(col1), sum(col3), min(col4) FROM foo
+   * (7) SELECT DISTINCT(DaysSinceEpoch), 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   *
+   * SQL versions of the above queries:
+   *
+   * (1) SELECT sum(col1), min(col2), DISTINCT col3, col4
+   * (2) SELECT col1, col2, DISTINCT col3 FROM foo
+   * (3) SELECT DISTINCT col1, col2, DISTINCT col3 FROM foo
+   * (4) SELECT timeConvert(DaysSinceEpoch,'DAYS','SECONDS'), DISTINCT 
DaysSinceEpoch FROM foo
+   *
+   * 1, 2, 3 and 4 will still not be supported in compliance with SQL
+   *
+   * (5) SELECT DISTINCT col1, col2, col3 FROM foo
+   * will be supported as it effectively becomes a multi column distinct
+   *
+   * (6) SELECT DISTINCT col1, sum(col3), min(col4) FROM foo
+   * although a valid SQL syntax for multi column distinct, if we decide to 
support
+   * them, we will have to do sum and min as transform functions which is not 
the case today.
+   * In any case, not a helpful query.
+   *
+   * (7) SELECT DISTINCT DaysSinceEpoch, 
timeConvert(DaysSinceEpoch,'DAYS','SECONDS') FROM foo
+   * again a valid SQL syntax for multi column distinct, we can support this 
since timeConvert
+   * is a valid supported transform function.
+   */
+  private void validate() {
+boolean identifierPresent = false;
+boolean distinctPresent = false;
+boolean functionPresent = false;
+if (hasChildren()) {
+  for (AstNode child : getChildren()) {
+if (child instanceof OutputColumnAstNode) {
 
 Review comment:
   Can child be other type of AstNode?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org