[GitHub] drill issue #519: DRILL-4530: Optimize partition pruning with metadata cachi...

2016-07-12 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/519
  
Other than prior comments, the pruning logic looks good to me. 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70532442
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java
 ---
@@ -211,9 +217,76 @@ public void testNoSupportedError() throws Exception {
 .go();
   }
 
+  @Test // DRILL-4530
+  public void testDrill4530_1() throws Exception {
+// create metadata cache
+test(String.format("refresh table metadata dfs_test.`%s/%s`", 
getDfsTestTmpSchemaLocation(), tableName2));
+checkForMetadataFile(tableName2);
+
+// run query and check correctness
+String query1 = String.format("select dir0, dir1, o_custkey, 
o_orderdate from dfs_test.`%s/%s` " +
+" where dir0=1995 and dir1='Q3'",
+getDfsTestTmpSchemaLocation(), tableName2);
+int expectedRowCount = 20;
+int expectedNumFiles = 2;
+
+int actualRowCount = testSql(query1);
+assertEquals(expectedRowCount, actualRowCount);
+String numFilesPattern = "numFiles=" + expectedNumFiles;
+String usedMetaPattern = "usedMetadataFile=true";
+String cacheFileRootPattern = String.format("%s/%s/1995/Q3", 
getDfsTestTmpSchemaLocation(), tableName2);
--- End diff --

The verification of cacheFileRootPattern probably need put "cacheFileRoot=" 
as the prefix. Otherwise,  the list of files in GroupScan will always find a 
match for cacheFileRoot, right?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70529269
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -143,6 +144,11 @@ public static final RelOptRule 
getDirFilterOnScan(OptimizerRulesContext optimize
   }
 
   protected void doOnMatch(RelOptRuleCall call, Filter filterRel, Project 
projectRel, TableScan scanRel) {
+if (wasAllPartitionsPruned) {
--- End diff --

I feel this flag may lead to scenarios where pruning does not happen when 
it should.  The flag is PruneScanRule's internal variable. The same rule could 
be applied to pruning for multiple tables.  So, if PruneScanRule turns on this 
flag for T1, then we might skip pruning logic for T2 later on, when this rule 
is fired.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70527515
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/ParquetPartitionDescriptor.java
 ---
@@ -130,13 +131,13 @@ protected void createPartitionSublists() {
   }
 
   @Override
-  public TableScan createTableScan(List 
newPartitionLocation) throws Exception {
+  public TableScan createTableScan(List 
newPartitionLocation, String cacheFileRoot) throws Exception {
--- End diff --

ParquetPartitionDescriptor does not supportsSinglePartOptimization. Should 
it here throws unsupportedException, and implement 
createTableScan(newPartitionLocation) only? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70524663
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
 ---
@@ -212,8 +223,12 @@ public TableScan 
createTableScan(List newPartitionLocation) t
 }
 
 if (scanRel instanceof DrillScanRel) {
-  final FileSelection newFileSelection = new FileSelection(null, 
newFiles, getBaseTableLocation());
-  final FileGroupScan newGroupScan = 
((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection);
+//  final FormatSelection formatSelection = 
(FormatSelection)((DynamicDrillTable)scanRel.getTable()).getSelection();
+  final FormatSelection formatSelection = 
(FormatSelection)table.getSelection();
+  final FileSelection newFileSelection = new FileSelection(null, 
newFiles, getBaseTableLocation(),
+  cacheFileRoot, formatSelection.getSelection().getDirStatus());
+  final FileGroupScan newGroupScan =
+  
((FileGroupScan)((DrillScanRel)scanRel).getGroupScan()).clone(newFileSelection, 
cacheFileRoot);
--- End diff --

This newFileSelection already has cacheFileRoot.  Do you think clone() only 
need take FileSelection, since FileSelection has cacheFileRoot infor?




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70514652
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -269,13 +283,54 @@ protected void doOnMatch(RelOptRuleCall call, Filter 
filterRel, Project projectR
 int recordCount = 0;
 int qualifiedCount = 0;
 
-// Inner loop: within each batch iterate over the 
PartitionLocations
-for(PartitionLocation part: partitions){
-  if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1){
-newPartitions.add(part);
-qualifiedCount++;
+if (checkForSingle &&
+partitions.get(0).isCompositePartition() /* apply single 
partition check only for composite partitions */) {
+  // Inner loop: within each batch iterate over the 
PartitionLocations
+  for (PartitionLocation part : partitions) {
--- End diff --

yes,  it's ok to put the refactoring effort as an enhancement JIR. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r70514381
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
 ---
@@ -607,7 +607,14 @@ public long getRowCount() {
   fileSet = Sets.newHashSet(fileNames);
 }
 
-if (fileNames.isEmpty()) {
+List finalFileNames;
+if (fileSet != null) {
+  finalFileNames = Lists.newArrayList(fileSet);
--- End diff --

Can we use fileNames only? From Line 589 / 607, seems fileSet is assigned 
from fileNames; seems they are same under two ELSE branches. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #543: DRILL-4768: Fix leaking hive meta store connection ...

2016-07-12 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/543#discussion_r70490413
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
 ---
@@ -129,7 +132,7 @@ public DrillHiveMetaStoreClient run() throws Exception {
*/
   public static DrillHiveMetaStoreClient 
createNonCloseableClientWithCaching(final HiveConf hiveConf)
--- End diff --

good catch. will modify. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #436: DRILL-4514 : Add describe schema command

2016-07-12 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/436
  
LGTM. 

+1

Thanks for the PR, @arina-ielchiieva ! 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #436: DRILL-4514 : Add describe schema comm...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r70379773
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
 ---
@@ -0,0 +1,129 @@
+/**
+ * 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.drill.exec.planner.sql.handlers;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.SerializableString;
+import com.fasterxml.jackson.core.io.CharacterEscapes;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlDescribeSchema;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+
+import java.util.List;
+import java.util.Map;
+
+import static 
com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
+public class DescribeSchemaHandler extends DefaultSqlHandler {
+
+  public DescribeSchemaHandler(SqlHandlerConfig config) {
+super(config);
+  }
+
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(DescribeSchemaHandler.class);
+  private static final ObjectMapper mapper = new ObjectMapper(new 
ObjectMapper().getFactory().setCharacterEscapes(new CharacterEscapes() {
+@Override
+public int[] getEscapeCodesForAscii() {
+  // add standard set of escaping characters
+  int[] esc = CharacterEscapes.standardAsciiEscapesForJSON();
+  // don't escape backslash (not to corrupt windows path)
+  esc['\\'] = CharacterEscapes.ESCAPE_NONE;
+  return esc;
+}
+
+@Override
+public SerializableString getEscapeSequence(int i) {
+  // no further escaping (beyond ASCII chars) needed
+  return null;
+}
+  })).enable(INDENT_OUTPUT);
+
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) {
+SqlIdentifier schema = ((SqlDescribeSchema) sqlNode).getSchema();
+SchemaPlus drillSchema = 
SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), 
schema.names);
+
+if (drillSchema != null) {
+  StoragePlugin storagePlugin;
+  try {
+storagePlugin = 
context.getStorage().getPlugin(schema.names.get(0));
+  } catch (ExecutionSetupException e) {
+throw UserException.validationError()
+.message("Failure while retrieving storage plugin", e)
+.build(logger);
+  }
+  String properties;
+  try {
+properties = getPropertiesAsJsonString(schema.names, 
storagePlugin.getConfig());
+  } catch (JsonProcessingException e) {
+throw UserException.parseError()
+.message("Error while trying to convert storage config to json 
string")
+.build(logger);
+  }
+  return DirectPlan.createDirectPlan(context, new 
DescribeSchemaResult(Joiner.on(".").join(schema.names), properties));
+}
+
+throw UserException.validationError()
+  .message(String.format("Invalid schema name [%s]", 
Joiner.on(".").join(schema.names)))
+  .build(logger);
+  }
+
+  /**
+   * Converts schema config properties to json st

[GitHub] drill pull request #436: DRILL-4514 : Add describe schema comm...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r70379333
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
 ---
@@ -0,0 +1,129 @@
+/**
+ * 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.drill.exec.planner.sql.handlers;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.SerializableString;
+import com.fasterxml.jackson.core.io.CharacterEscapes;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlDescribeSchema;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+
+import java.util.List;
+import java.util.Map;
+
+import static 
com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
+public class DescribeSchemaHandler extends DefaultSqlHandler {
+
+  public DescribeSchemaHandler(SqlHandlerConfig config) {
+super(config);
+  }
+
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(DescribeSchemaHandler.class);
+  private static final ObjectMapper mapper = new ObjectMapper(new 
ObjectMapper().getFactory().setCharacterEscapes(new CharacterEscapes() {
+@Override
+public int[] getEscapeCodesForAscii() {
+  // add standard set of escaping characters
+  int[] esc = CharacterEscapes.standardAsciiEscapesForJSON();
+  // don't escape backslash (not to corrupt windows path)
+  esc['\\'] = CharacterEscapes.ESCAPE_NONE;
+  return esc;
+}
+
+@Override
+public SerializableString getEscapeSequence(int i) {
+  // no further escaping (beyond ASCII chars) needed
+  return null;
+}
+  })).enable(INDENT_OUTPUT);
+
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) {
+SqlIdentifier schema = ((SqlDescribeSchema) sqlNode).getSchema();
+SchemaPlus drillSchema = 
SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), 
schema.names);
+
+if (drillSchema != null) {
+  StoragePlugin storagePlugin;
+  try {
+storagePlugin = 
context.getStorage().getPlugin(schema.names.get(0));
+  } catch (ExecutionSetupException e) {
+throw UserException.validationError()
+.message("Failure while retrieving storage plugin", e)
+.build(logger);
+  }
+  String properties;
+  try {
+properties = getPropertiesAsJsonString(schema.names, 
storagePlugin.getConfig());
+  } catch (JsonProcessingException e) {
+throw UserException.parseError()
--- End diff --

Same as above. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #436: DRILL-4514 : Add describe schema comm...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r70379318
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
 ---
@@ -0,0 +1,129 @@
+/**
+ * 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.drill.exec.planner.sql.handlers;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.SerializableString;
+import com.fasterxml.jackson.core.io.CharacterEscapes;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlDescribeSchema;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+
+import java.util.List;
+import java.util.Map;
+
+import static 
com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
+public class DescribeSchemaHandler extends DefaultSqlHandler {
+
+  public DescribeSchemaHandler(SqlHandlerConfig config) {
+super(config);
+  }
+
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(DescribeSchemaHandler.class);
+  private static final ObjectMapper mapper = new ObjectMapper(new 
ObjectMapper().getFactory().setCharacterEscapes(new CharacterEscapes() {
+@Override
+public int[] getEscapeCodesForAscii() {
+  // add standard set of escaping characters
+  int[] esc = CharacterEscapes.standardAsciiEscapesForJSON();
+  // don't escape backslash (not to corrupt windows path)
+  esc['\\'] = CharacterEscapes.ESCAPE_NONE;
+  return esc;
+}
+
+@Override
+public SerializableString getEscapeSequence(int i) {
+  // no further escaping (beyond ASCII chars) needed
+  return null;
+}
+  })).enable(INDENT_OUTPUT);
+
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) {
+SqlIdentifier schema = ((SqlDescribeSchema) sqlNode).getSchema();
+SchemaPlus drillSchema = 
SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), 
schema.names);
+
+if (drillSchema != null) {
+  StoragePlugin storagePlugin;
+  try {
+storagePlugin = 
context.getStorage().getPlugin(schema.names.get(0));
+  } catch (ExecutionSetupException e) {
+throw UserException.validationError()
--- End diff --

Any reason why you treat ExecutionSetupException as 
UserException:validationError? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #436: DRILL-4514 : Add describe schema comm...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r70378389
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DescribeSchemaHandler.java
 ---
@@ -0,0 +1,129 @@
+/**
+ * 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.drill.exec.planner.sql.handlers;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.SerializableString;
+import com.fasterxml.jackson.core.io.CharacterEscapes;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Joiner;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.apache.drill.exec.planner.sql.SchemaUtilites;
+import org.apache.drill.exec.planner.sql.parser.SqlDescribeSchema;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.apache.drill.exec.store.dfs.FileSystemSchemaFactory;
+import org.apache.drill.exec.store.dfs.WorkspaceConfig;
+
+import java.util.List;
+import java.util.Map;
+
+import static 
com.fasterxml.jackson.databind.SerializationFeature.INDENT_OUTPUT;
+
+public class DescribeSchemaHandler extends DefaultSqlHandler {
+
+  public DescribeSchemaHandler(SqlHandlerConfig config) {
+super(config);
+  }
+
+  private static final org.slf4j.Logger logger = 
org.slf4j.LoggerFactory.getLogger(DescribeSchemaHandler.class);
+  private static final ObjectMapper mapper = new ObjectMapper(new 
ObjectMapper().getFactory().setCharacterEscapes(new CharacterEscapes() {
+@Override
+public int[] getEscapeCodesForAscii() {
+  // add standard set of escaping characters
+  int[] esc = CharacterEscapes.standardAsciiEscapesForJSON();
+  // don't escape backslash (not to corrupt windows path)
+  esc['\\'] = CharacterEscapes.ESCAPE_NONE;
+  return esc;
+}
+
+@Override
+public SerializableString getEscapeSequence(int i) {
+  // no further escaping (beyond ASCII chars) needed
+  return null;
+}
+  })).enable(INDENT_OUTPUT);
+
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) {
+SqlIdentifier schema = ((SqlDescribeSchema) sqlNode).getSchema();
+SchemaPlus drillSchema = 
SchemaUtilites.findSchema(config.getConverter().getDefaultSchema(), 
schema.names);
+
+if (drillSchema != null) {
+  StoragePlugin storagePlugin;
+  try {
+storagePlugin = 
context.getStorage().getPlugin(schema.names.get(0));
+  } catch (ExecutionSetupException e) {
+throw UserException.validationError()
+.message("Failure while retrieving storage plugin", e)
+.build(logger);
+  }
+  String properties;
+  try {
+properties = getPropertiesAsJsonString(schema.names, 
storagePlugin.getConfig());
+  } catch (JsonProcessingException e) {
+throw UserException.parseError()
+.message("Error while trying to convert storage config to json 
string")
+.build(logger);
+  }
+  return DirectPlan.createDirectPlan(context, new 
DescribeSchemaResult(Joiner.on(".").join(schema.names), properties));
+}
+
+throw UserException.validationError()
+  .message(String.format("Invalid schema name [%s]", 
Joiner.on(".").join(schema.names)))
+  .build(logger);
+  }
+
+  /**
+   * Converts schema config properties to json st

[GitHub] drill pull request #436: DRILL-4514 : Add describe schema comm...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r70377349
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestInfoSchema.java ---
@@ -351,4 +359,51 @@ public void showFilesWithDefaultSchema() throws 
Exception{
 test("USE dfs_test.`default`");
 test("SHOW FILES FROM `/tmp`");
   }
+
+  @Test
+  public void describeSchemaSyntax() throws Exception {
+test("describe schema dfs_test");
+test("describe schema dfs_test.`default`");
+test("describe database dfs_test.`default`");
+  }
+
+  @Test
+  public void describeSchemaOutputForDefaultWorkspace() throws Exception {
+String properties = 
Resources.toString(Resources.getResource("describe_schema_output.json"), 
Charsets.UTF_8).replace("\r", "");
+testBuilder()
+.sqlQuery("describe schema dfs_test")
+.unOrdered()
+.baselineColumns("schema", "properties")
+.baselineValues("dfs_test", properties)
--- End diff --

I'm not sure if it's a good idea to put the complete "properties" as the 
expected output in unit test. Doing that makes this unit test prone to failure, 
in case we change StoragePlugin interface, and add / delete contents in the 
schema properties. This is similar to Explain Plan output. If we capture 
everything, it becomes annoying in the future, since any slight change to the 
planer might fail the unit test.

In stead of capturing everything, can you just verify some "stable" part of 
properties, like "location" etc, to make this unit test less likely to fail in 
the future?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #543: DRILL-4768: Fix leaking hive meta store connection ...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/543#discussion_r70361935
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
 ---
@@ -209,8 +213,10 @@ private DrillHiveMetaStoreClient(final HiveConf 
hiveConf) throws MetaException {
   throws TException {
 try {
   return mClient.getAllTables(dbName);
+} catch (MetaException | UnknownDBException e) {
+  throw e;
 } catch (TException e) {
-  logger.warn("Failure while attempting to get hive tables", e);
+  logger.warn("Failure while attempting to get hive tables. Retries 
once.", e);
--- End diff --

Add code to check if close() hits exception. In case close() fails with 
exception, we log the exception and continue to reconnect(). This may leak 
connection. 

However, we have to maintain hive metastore connection. Otherwise, next 
hive query may fail, and the only way to continue is either re-enable hive 
storage plugin (which may then leak a connection DRILL-4774), or re-start 
drillbit.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #543: DRILL-4768: Fix leaking hive meta store connection ...

2016-07-11 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/543#discussion_r70360847
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/DrillHiveMetaStoreClient.java
 ---
@@ -209,8 +213,10 @@ private DrillHiveMetaStoreClient(final HiveConf 
hiveConf) throws MetaException {
   throws TException {
 try {
   return mClient.getAllTables(dbName);
+} catch (MetaException | UnknownDBException e) {
+  throw e;
 } catch (TException e) {
-  logger.warn("Failure while attempting to get hive tables", e);
+  logger.warn("Failure while attempting to get hive tables. Retries 
once.", e);
--- End diff --

Thanks for the suggestion. I modified the code by calling close() 
explicitly. However, to make it work, we also have to change the client from 
nonClosable to Closable(), by removing the overriden close() method. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #543: DRILL-4768: Fix leaking hive meta store connection in Dril...

2016-07-11 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/543
  
Revise the patch based on @vkorukanti 's comments.

Pls let me know if you have any further comment. Otherwise, I'll run 
regression and merge the patch.  Thanks. 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #518: DRILL-4653.json - Malformed JSON should not stop the entir...

2016-07-11 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/518
  
@chunhui-shi , I saw you made comments days ago. Can you pls take a look at 
the new patch to see if it addressed your comment? thx. 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #543: DRILL-4768: Fix leaking hive meta store connection in Dril...

2016-07-11 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/543
  
@sudheeshkatkam and @vkorukanti , can one of you review this PR? Thanks!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #543: DRILL-4768: Fix leaking hive meta store connection ...

2016-07-11 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/543

DRILL-4768: Fix leaking hive meta store connection in Drill's hive me…

…tastore client call.

do not call reconnect if the connection is still alive and the error is 
caused by either UnknownTableException or access error.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4768

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/543.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #543


commit 20a46950b37d05b7e5b0500f0d195ac7f418fe8d
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-07-08T04:48:53Z

DRILL-4768: Fix leaking hive meta store connection in Drill's hive 
metastore client call.

do not call reconnect if the connection is still alive and the error is 
caused by either UnknownTableException or access error.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #436: DRILL-4514 : Add describe schema command

2016-07-08 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/436
  
The new output looks better. 

I checked hive's  command. Seems it lists schema name 
and its relevant properties. Does it make sense to add schema | database name 
in the output?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #520: DRILL-3510: Add ANSI_QUOTES option so that Drill's ...

2016-07-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/520#discussion_r69997781
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
 ---
@@ -36,6 +42,7 @@
  */
 class DrillDatabaseMetaDataImpl extends AvaticaDatabaseMetaData
--- End diff --

Also, does Drill allow user to specify the ANSI_QUOTE option in JDBC 
connection string? If not, that means user does not have a way to change 
ANSI_QOUTE option if they connect through JDBC?  If so, then lots of BI tool 
will not benefit from this patch, if they want to use double quotes in their 
sql statement. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #520: DRILL-3510: Add ANSI_QUOTES option so that Drill's ...

2016-07-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/520#discussion_r69997455
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
 ---
@@ -219,11 +226,26 @@ public boolean storesMixedCaseQuotedIdentifiers() 
throws SQLException {
 return super.storesMixedCaseQuotedIdentifiers();
   }
 
-  // TODO(DRILL-3510):  Update when Drill accepts standard SQL's double 
quote.
   @Override
   public String getIdentifierQuoteString() throws SQLException {
 throwIfClosed();
-return "`";
+boolean systemOption = false;
+boolean sessionOption = false;
+String sql = "select type, bool_val from sys.options where name = 
'parser.ansi_quotes'";
+ResultSet rs = executeSql(sql);
+while (rs.next()) {
+  if (rs.getString(1).equals("SYSTEM")) {
+systemOption = rs.getBoolean(2);
+  }
+  if (rs.getString(1).equals("SESSION")) {
+sessionOption = rs.getBoolean(2);
+  }
+}
+if (systemOption || sessionOption) {
--- End diff --

If session option is turned off, but system options is turned on for 
ansi_quote, do we still return DOULBE_QUOTE?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #520: DRILL-3510: Add ANSI_QUOTES option so that Drill's ...

2016-07-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/520#discussion_r69997336
  
--- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillDatabaseMetaDataImpl.java
 ---
@@ -219,11 +226,26 @@ public boolean storesMixedCaseQuotedIdentifiers() 
throws SQLException {
 return super.storesMixedCaseQuotedIdentifiers();
   }
 
-  // TODO(DRILL-3510):  Update when Drill accepts standard SQL's double 
quote.
   @Override
   public String getIdentifierQuoteString() throws SQLException {
 throwIfClosed();
-return "`";
+boolean systemOption = false;
+boolean sessionOption = false;
+String sql = "select type, bool_val from sys.options where name = 
'parser.ansi_quotes'";
--- End diff --

Define a string constant for the option name, and use it across the code. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #520: DRILL-3510: Add ANSI_QUOTES option so that Drill's ...

2016-07-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/520#discussion_r69990725
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
 ---
@@ -61,7 +61,8 @@ public static PhysicalPlan getPlan(QueryContext context, 
String sql, Pointer

[GitHub] drill pull request #520: DRILL-3510: Add ANSI_QUOTES option so that Drill's ...

2016-07-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/520#discussion_r69990123
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/ExecConstants.java ---
@@ -274,6 +274,9 @@
   String ENABLE_BULK_LOAD_TABLE_LIST_KEY = 
"exec.enable_bulk_load_table_list";
   BooleanValidator ENABLE_BULK_LOAD_TABLE_LIST = new 
BooleanValidator(ENABLE_BULK_LOAD_TABLE_LIST_KEY, false);
 
+  String ANSI_QUOTES_KEY = "parser.ansi_quotes";
--- End diff --

this option had better go to PlannerSettings.java, since it's not an option 
for exec. Also, please follow existing naming convention for new option name. 
For this case, how about `planner.parser.ansi_quotes`? 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-28 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r68814702
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java 
---
@@ -47,16 +47,25 @@
   private List statuses;
 
   public List files;
+  /**
+   * root path for the selections
+   */
   public final String selectionRoot;
+  /**
+   * root path for the metadata cache file (if any)
+   */
+  public final String cacheFileRoot;
--- End diff --

Thanks for the explanation. It makes sense and I do see why updating 
selectionRoot would cause problems for the queries you mentioned. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #515: DRILL-4707: Fix memory leak or incorrect query result in c...

2016-06-27 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/515
  
I cherry-picked CALCITE-528 from Calcite master branch, and put it into 
Drill forked version. 
@amansinha100 , could you please review this PR? thanks!




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #521: DRILL-4715: Fix java compilation error in run-time generat...

2016-06-23 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/521
  
@sudheeshkatkam , do you want to take a quick look and see if I have 
addressed your comments? Thanks!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68148042
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java
 ---
@@ -186,4 +186,9 @@ public boolean isFunctionComplexOutput(String name) {
 }
 return false;
   }
+
+  public OptionManager getOptionManager() {
--- End diff --

Removed. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68148064
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
 ---
@@ -17,18 +17,16 @@
  */
 package org.apache.drill.exec.physical.impl;
 
+import static org.apache.drill.exec.RunRootExec.c;
--- End diff --

removed 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68148021
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---
@@ -247,11 +275,13 @@ void flushCode() {
   outer._throws(SchemaChangeException.class);
 
   int methodIndex = 0;
-  int blocksInMethod = 0;
+  int exprsInMethod = 0;
   boolean isVoidMethod = method.getReturnType() == void.class;
-  for(JBlock b : blocks[i++]) {
+  for(SizedJBlock sb : blocks[i++]) {
+JBlock b = sb.getBlock();
 if(!b.isEmpty()) {
-  if (blocksInMethod > MAX_BLOCKS_IN_FUNCTION) {
+  if (optionManager != null &&
+  exprsInMethod > 
optionManager.getOption(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE).num_val) {
--- End diff --

Done.  thx. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68147991
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---
@@ -215,22 +218,47 @@ public JVar 
declareVectorValueSetupAndMember(DirectExpression batchName, TypedFi
 return vv;
   }
 
+  public enum BlkCreateMode {
+TRUE,  // Create new block
+FALSE, // Do not create block; put into existing block.
+TRUE_IF_BOUND // Create new block only if # of expressions added hit 
upper-bound (ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE)
+  }
+
   public HoldingContainer addExpr(LogicalExpression ex) {
-return addExpr(ex, true);
+// default behavior is always to put expression into new block.
+return addExpr(ex, BlkCreateMode.TRUE);
   }
 
-  public HoldingContainer addExpr(LogicalExpression ex, boolean rotate) {
-//logger.debug("Adding next write {}", ex);
-if (rotate) {
-  rotateBlock();
+  public HoldingContainer addExpr(LogicalExpression ex, BlkCreateMode 
mode) {
+if (mode == BlkCreateMode.TRUE || mode == BlkCreateMode.TRUE_IF_BOUND) 
{
+  rotateBlock(mode);
 }
+
+for (LinkedList b : blocks) {
+  b.getLast().incCounter();
+}
+
 return evaluationVisitor.addExpr(ex, this);
   }
 
   public void rotateBlock() {
-evaluationVisitor.previousExpressions.clear();
-for (LinkedList b : blocks) {
-  b.add(new JBlock(true, true));
+// default behavior is always to create new block.
+rotateBlock(BlkCreateMode.TRUE);
+  }
+
+  private void rotateBlock(BlkCreateMode mode) {
+boolean blockRotated = false;
+for (LinkedList b : blocks) {
+  if (mode == BlkCreateMode.TRUE ||
+  (mode == BlkCreateMode.TRUE_IF_BOUND &&
+optionManager != null &&
+b.getLast().getCount() > 
optionManager.getOption(ExecConstants.CODE_GEN_EXP_IN_METHOD_SIZE).num_val)) {
--- End diff --

Done. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68102519
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/SizedJBlock.java ---
@@ -0,0 +1,48 @@
+/**
+ * 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.drill.exec.expr;
+
+import com.sun.codemodel.JBlock;
+
+/**
+ * Uses this class to keep track # of Drill Logical Expressions that are
+ * put to JBlock.
+ */
+public class SizedJBlock {
+  private final JBlock block;
--- End diff --

Added comment. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68102435
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---
@@ -215,22 +219,47 @@ public JVar 
declareVectorValueSetupAndMember(DirectExpression batchName, TypedFi
 return vv;
   }
 
+  public enum RotateMode {
--- End diff --

Rename it to BlcCreationMode. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/521#discussion_r68102305
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/ClassGenerator.java ---
@@ -86,14 +88,16 @@ public static MappingSet getDefaultMapping() {
   }
 
   @SuppressWarnings("unchecked")
-  ClassGenerator(CodeGenerator codeGenerator, MappingSet mappingSet, 
SignatureHolder signature, EvaluationVisitor eval, JDefinedClass clazz, 
JCodeModel model) throws JClassAlreadyExistsException {
+  ClassGenerator(CodeGenerator codeGenerator, MappingSet mappingSet, 
SignatureHolder signature, EvaluationVisitor eval, JDefinedClass clazz, 
JCodeModel model, OptionManager optionManager) throws 
JClassAlreadyExistsException {
 this.codeGenerator = codeGenerator;
 this.clazz = clazz;
 this.mappings = mappingSet;
 this.sig = signature;
 this.evaluationVisitor = eval;
 this.model = model;
-blocks = (LinkedList[]) new LinkedList[sig.size()];
+this.optionManager = optionManager;
--- End diff --

Revise the code to pass in session option manager. 

Also refactor some existing unit tests to pass in option manager to unit 
tests. 

@sudheeshkatkam , could you please take a quick look to see if it addresses 
your comment? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #531: DRILL-4733: max(dir0) reading more columns than nec...

2016-06-22 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/531#discussion_r68070931
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/TestImplicitFileColumns.java
 ---
@@ -110,4 +111,20 @@ public void testImplicitColumnsForParquet() throws 
Exception {
 .go();
   }
 
+  @Test // DRILL-4733
+  public void testMultilevelParquetWithSchemaChange() throws Exception {
+try {
+  test("alter session set `planner.enable_decimal_data_type` = true");
+  testBuilder()
+  .sqlQuery(String.format("select max(dir0) as max_dir from 
dfs_test.`%s/src/test/resources/multilevel/parquetWithSchemaChange`",
+  TestTools.getWorkingPath()))
+  .unOrdered()
+  .baselineColumns("max_dir")
+  .baselineValues("voter50.parquet")
--- End diff --

Got it. "voter50.parquet" is actually a folder name. I thought it's a 
parquet file. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67716686
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java 
---
@@ -47,16 +47,25 @@
   private List statuses;
 
   public List files;
+  /**
+   * root path for the selections
+   */
   public final String selectionRoot;
+  /**
+   * root path for the metadata cache file (if any)
+   */
+  public final String cacheFileRoot;
--- End diff --

When singlePartitionOpt is applied,  is it possible to update selectionRoot 
to be cacheFileRoot? That is, we do not maintain cacheFileRoot separately. In 
stead, a FileSelection with updated selectionRoot is used when 
singlePartitionOpt is applied. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67711244
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -269,13 +283,54 @@ protected void doOnMatch(RelOptRuleCall call, Filter 
filterRel, Project projectR
 int recordCount = 0;
 int qualifiedCount = 0;
 
-// Inner loop: within each batch iterate over the 
PartitionLocations
-for(PartitionLocation part: partitions){
-  if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1){
-newPartitions.add(part);
-qualifiedCount++;
+if (checkForSingle &&
+partitions.get(0).isCompositePartition() /* apply single 
partition check only for composite partitions */) {
+  // Inner loop: within each batch iterate over the 
PartitionLocations
+  for (PartitionLocation part : partitions) {
+assert part.isCompositePartition();
+if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1) {
+  newPartitions.add(part);
+  if (isSinglePartition) { // only need to do this if we are 
already single partition
+// compose the array of partition values for the 
directories that are referenced by filter:
+// e.g suppose the dir hierarchy is year/quarter/month and 
the query is:
+// SELECT * FROM T WHERE dir0=2015 AND dir1 = 'Q1',
+// then for 2015/Q1/Feb, this will have ['2015', 'Q1', 
null]
--- End diff --

For WHERE condition   dir0=2015 and dir2 = 'Jan', if the dataset happens to 
have only one 'Jan' under '2015' directory, will this qualify for 
singlePartitionOpt?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67709299
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -320,7 +377,17 @@ protected void doOnMatch(RelOptRuleCall call, Filter 
filterRel, Project projectR
   condition = condition.accept(reverseVisitor);
   pruneCondition = pruneCondition.accept(reverseVisitor);
 
-  RelNode inputRel = descriptor.createTableScan(newPartitions);
+  String cacheFileRoot = null;
+  if (checkForSingle && isSinglePartition) {
+// if metadata cache file could potentially be used, then assign a 
proper cacheFileRoot
+String path = "";
+for (int j = 0; j <= maxIndex; j++) {
+  path += "/" + spInfo[j];
--- End diff --

Related to Line 313, here we do not check spInfo[j] == null ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-17 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67567993
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -269,13 +283,54 @@ protected void doOnMatch(RelOptRuleCall call, Filter 
filterRel, Project projectR
 int recordCount = 0;
 int qualifiedCount = 0;
 
-// Inner loop: within each batch iterate over the 
PartitionLocations
-for(PartitionLocation part: partitions){
-  if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1){
-newPartitions.add(part);
-qualifiedCount++;
+if (checkForSingle &&
+partitions.get(0).isCompositePartition() /* apply single 
partition check only for composite partitions */) {
+  // Inner loop: within each batch iterate over the 
PartitionLocations
+  for (PartitionLocation part : partitions) {
+assert part.isCompositePartition();
+if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1) {
+  newPartitions.add(part);
+  if (isSinglePartition) { // only need to do this if we are 
already single partition
+// compose the array of partition values for the 
directories that are referenced by filter:
+// e.g suppose the dir hierarchy is year/quarter/month and 
the query is:
+// SELECT * FROM T WHERE dir0=2015 AND dir1 = 'Q1',
+// then for 2015/Q1/Feb, this will have ['2015', 'Q1', 
null]
+// Note that we are not using the PartitionLocation here 
but composing a different list because
+// we are only interested in the directory columns that 
are referenced in the filter condition. not
+// the SELECT list or other parts of the query.
+Pair<String[], Integer> p = 
composePartition(referencedDirsBitSet, partitionMap, vectors, recordCount);
+String[] parts = p.getLeft();
+int tmpIndex = p.getRight();
+if (spInfo == null) {
+  spInfo = parts;
+  maxIndex = tmpIndex;
+} else if (maxIndex != tmpIndex) {
+  isSinglePartition = false;
+  break;
+} else {
+  // we only want to compare until the maxIndex inclusive 
since subsequent values would be null
+  for (int j = 0; j <= maxIndex; j++) {
+if (spInfo[j] == null // prefixes should be non-null
--- End diff --

Form Line 305-306, spInfo and maxIndex are in sync. Why will we have 
spInfo[j] == null, when j <= maxIndex? I thought maxIndex is obtained such that 
element in spInfo is not null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-17 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67567703
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HivePartitionDescriptor.java
 ---
@@ -151,7 +152,7 @@ protected void createPartitionSublists() {
   }
 
   @Override
-  public TableScan createTableScan(List newPartitions) 
throws Exception {
+  public TableScan createTableScan(List newPartitions, 
String cacheFileRoot) throws Exception {
--- End diff --

From the comment, cacheFileRoot could be null. But is it possible to not 
adding this parameter to HivePartitionDescriptor? After all, cacheFileRoot will 
not be applied to hive partition. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #519: DRILL-4530: Optimize partition pruning with metadat...

2016-06-17 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/519#discussion_r67567343
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/partition/PruneScanRule.java
 ---
@@ -269,13 +283,54 @@ protected void doOnMatch(RelOptRuleCall call, Filter 
filterRel, Project projectR
 int recordCount = 0;
 int qualifiedCount = 0;
 
-// Inner loop: within each batch iterate over the 
PartitionLocations
-for(PartitionLocation part: partitions){
-  if(!output.getAccessor().isNull(recordCount) && 
output.getAccessor().get(recordCount) == 1){
-newPartitions.add(part);
-qualifiedCount++;
+if (checkForSingle &&
+partitions.get(0).isCompositePartition() /* apply single 
partition check only for composite partitions */) {
+  // Inner loop: within each batch iterate over the 
PartitionLocations
+  for (PartitionLocation part : partitions) {
--- End diff --

Do you think it's possible to refactor this part of code such that only 
file system prune scan will have this logic?  This simplePartition optimization 
only applies to file system prune. Also, doOnMatch() method is getting bigger. 
Something like:  descriptor has supportsSinglePartOptimization(), and prune 
scan rule has doSinglePartOpt(), which is by default a no-op, and has 
implementation for file system prune rule.
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #512: Drill 4573 fix issue with unicode chars

2016-06-15 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/512
  
I did the following small changes to this PR, and I'm going to rebase and 
re-run the regression suite before merge this PR.

1) fix one bug which causes regression failure (IOBE).
2) add unit test with regexp function over non-ascii input
3) code clean up.

+1



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #521: DRILL-4715: Fix java compilation error in run-time generat...

2016-06-14 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/521
  
@jacques-n @amansinha100 , can you review this PR?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #521: DRILL-4715: Fix java compilation error in run-time ...

2016-06-14 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/521

DRILL-4715: Fix java compilation error in run-time generated code whe…

…n query has large number of expressions.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4715

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/521.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #521


commit 600b01dcaddc2e8e9cb95f728c263ebcfa7441e5
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-06-10T00:08:45Z

DRILL-4715: Fix java compilation error in run-time generated code when 
query has large number of expressions.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #512: Drill 4573 fix issue with unicode chars

2016-06-13 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/512
  
@jcmcote ,

Overall the patch looks good to me. Once you revise the code, I'll run the 
regression and merge the code if no problem is found. Thanks for your patch!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #512: Drill 4573 fix issue with unicode chars

2016-06-13 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/512#discussion_r66894981
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
 ---
@@ -114,6 +114,19 @@ public void testRegexpMatches() throws Exception {
   }
 
   @Test
+  public void testRegexpReplace() throws Exception {
+testBuilder()
+.sqlQuery("select regexp_replace(a, 'a|c', 'x') res1, 
regexp_replace(b, 'd', 'zzz') res2 " +
+  "from (values('abc', 'bcd'), ('bcd', 'abc')) as t(a,b)")
+.unOrdered()
+.baselineColumns("res1", "res2")
+.baselineValues("xbx", "bczzz")
+.baselineValues("bxd", "abc")
+.build()
+.run();
+  }
+
--- End diff --

that's probably Calcite uses 'ISO-8859-1', which does not cover the special 
'€'.

Maybe you can use the following string? It has non-ascii character, and is 
covered by 'ISO-8859-1'.

select regexp_matches('München', 'München') res3 from (values(1));

[1] https://en.wikipedia.org/wiki/ISO/IEC_8859-1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #512: Drill 4573 fix issue with unicode chars

2016-06-13 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/512#discussion_r66894665
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CharSequenceWrapper.java
 ---
@@ -17,13 +17,52 @@
  */
 package org.apache.drill.exec.expr.fn.impl;
 
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CoderResult;
+import java.util.regex.Matcher;
+
 import io.netty.buffer.DrillBuf;
 
+/**
+ * A CharSequence is a readable sequence of char values. This interface 
provides
+ * uniform, read-only access to many different kinds of char sequences. A 
char
+ * value represents a character in the Basic Multilingual Plane (BMP) or a
+ * surrogate. Refer to Unicode Character Representation for details.
+ * Specifically this implementation of the CharSequence adapts a Drill
+ * {@link DrillBuf} to the CharSequence. The implementation is meant to be
+ * re-used that is allocated once and then passed DrillBuf to adapt. This 
can be
+ * handy to exploit API that consume CharSequence avoiding the need to 
create
+ * string objects.
+ *
+ */
 public class CharSequenceWrapper implements CharSequence {
 
+// The adapted drill buffer (in the case of US-ASCII)
+private DrillBuf buffer;
+// The converted bytes in the case of non ASCII
+private CharBuffer charBuffer;
--- End diff --

Agreed. Make sense to use CharSequenceWrapper.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #512: Drill 4573 fix issue with unicode chars

2016-06-06 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/512#discussion_r65994802
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/exec/expr/fn/impl/TestStringFunctions.java
 ---
@@ -114,6 +114,19 @@ public void testRegexpMatches() throws Exception {
   }
 
   @Test
+  public void testRegexpReplace() throws Exception {
+testBuilder()
+.sqlQuery("select regexp_replace(a, 'a|c', 'x') res1, 
regexp_replace(b, 'd', 'zzz') res2 " +
+  "from (values('abc', 'bcd'), ('bcd', 'abc')) as t(a,b)")
+.unOrdered()
+.baselineColumns("res1", "res2")
+.baselineValues("xbx", "bczzz")
+.baselineValues("bxd", "abc")
+.build()
+.run();
+  }
+
--- End diff --

Can you add at least one case for the non-ASCII case, since one of the 
issue this PR tries to address is the incorrect result for regex function over 
non-ASCII input?  

A new unit test case will help verify this PR fix the problem, and make 
sure any new change will not regress this functionality. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request #515: DRILL-4707: Fix memory leak or incorrect query resu...

2016-06-06 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/515

DRILL-4707: Fix memory leak or incorrect query result in case two col…

…umn names are case-insensitive identical

Fix is mainly in CALCITE-528.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4707

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/515.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #515


commit 1bab184e862be5baa5f47804f506bdb56b35b374
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-06-06T00:37:22Z

DRILL-4707: Fix memory leak or incorrect query result in case two column 
names are case-insensitive identical

Fix is mainly in CALCITE-528.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill issue #512: Drill 4573 fix issue with unicode chars

2016-06-03 Thread jinfengni
Github user jinfengni commented on the issue:

https://github.com/apache/drill/pull/512
  
@jcmcote , thanks for the new PR. I'll take a look tomorrow, and let you 
know my feedback. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4693: Ensure final column re-ordering is...

2016-05-25 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/508#issuecomment-221749835
  
LGGM.

+1




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4573: Zero copy LIKE, REGEXP_MATCHES, SU...

2016-05-25 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/458#issuecomment-221648222
  
@jcmcote , just wanna to check the current status of your latest fix.  

FYI, you may consider using couple of existing methods [1] [2], when you 
decode the byte arrays as UDF8 characters, in CharaSequenceWrapper.


[1] 
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionUtil.java#L77

[2] 
https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionUtil.java#L28
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4514 : Add describe schema ...

2016-05-23 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r64299811
  
--- Diff: exec/java-exec/src/main/resources/bootstrap-storage-plugins.json 
---
@@ -6,11 +6,13 @@
   workspaces: {
 "root" : {
   location: "/",
-  writable: false
+  writable: false,
+  physicalLocation: "file:///"
--- End diff --

Looks like "physicalLocation" could be constructed as "connection" + 
"location" ?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4514 : Add describe schema ...

2016-05-23 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/436#discussion_r64299337
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceConfig.java
 ---
@@ -25,23 +25,27 @@
  *  - location which is a path.
  *  - writable flag to indicate whether the location supports creating new 
tables.
  *  - default storage format for new tables created in this workspace.
+ *  - physicalLocation is full path to workspace
  */
 @JsonIgnoreProperties(value = {"storageformat"})
 public class WorkspaceConfig {
 
   /** Default workspace is a root directory which supports read, but not 
write. */
-  public static final WorkspaceConfig DEFAULT = new WorkspaceConfig("/", 
false, null);
+  public static final WorkspaceConfig DEFAULT = new WorkspaceConfig("/", 
false, null, null);
 
   private final String location;
   private final boolean writable;
   private final String defaultInputFormat;
+  private final String physicalLocation;
 
   public WorkspaceConfig(@JsonProperty("location") String location,
  @JsonProperty("writable") boolean writable,
- @JsonProperty("defaultInputFormat") String 
defaultInputFormat) {
+ @JsonProperty("defaultInputFormat") String 
defaultInputFormat,
+ @JsonProperty("physicalLocation") String 
physicalLocation) {
 this.location = location;
 this.writable = writable;
 this.defaultInputFormat = defaultInputFormat;
+this.physicalLocation = physicalLocation;
--- End diff --

WorkspaceConfig already has "location". Why do you prefer adding a new 
"physicalLocation"? Is it possible to use "location" in stead?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4679: When convert() functions are prese...

2016-05-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/504#discussion_r64064412
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
 ---
@@ -136,6 +145,10 @@ public VectorContainer getOutgoingContainer() {
 
   @Override
   protected IterOutcome doWork() {
+if (wasNone) {
+  return IterOutcome.NONE;
+}
+
 int incomingRecordCount = incoming.getRecordCount();
 
 if (first && incomingRecordCount == 0) {
--- End diff --

The new logic will handle the case for Project's first outgoing batch. Not 
sure whether Drill works properly after the first batch getting data and 
building the schema, but the next incoming batch contains empty result. We may 
treat as a separate issue for further investigation. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4679: When convert() functions are prese...

2016-05-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/504#discussion_r64063077
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
 ---
@@ -146,6 +159,27 @@ protected IterOutcome doWork() {
   if (next == IterOutcome.OUT_OF_MEMORY) {
 outOfMemory = true;
 return next;
+  } else if (next == IterOutcome.NONE) {
+// since this is first batch and we already got a NONE, need 
to set up the schema
+
+//allocate vv in the allocationVectors.
+for (final ValueVector v : this.allocationVectors) {
--- End diff --

the allocation logic may use existing method doAlloc().



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4573: Zero copy LIKE, REGEXP_MATCHES, SU...

2016-05-19 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/458#issuecomment-220393770
  
@jcmcote , I opened DRILL-4688 to track this incorrect regression. Please 
use DRILL-4688 to submit either a patch or pull request to address this issue.

As part of the verification for this issue, QA plan to add a set of test 
suite to cover the cases of multi-byte characters.  Previously, there is no 
such test coverage and hence we are not able to catch this regression.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4573: Zero copy LIKE, REGEXP_MATCHES, SU...

2016-05-17 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/458#issuecomment-219858967
  
@jcmcote ,

(copy from my comment in DRILL-4573)

 I re-visited your first patch. Looks like that the change you made would 
cause incorrect result when the input string is a udf-8 with each character 
consisting of multiple bytes. In particular, the original implementation would 
encode the byte array with udf-8 (which is the default encoding in drill). 
However, in your CharSequenceWrapper, you will treat each byte as a character. 
This will cause incorrect result for case when a character is represented by >1 
bytes.

For instance, look at the following example, the first query of 
regexp_matches will produce wrong result. 
 
{code:sql}
select regexp_matches('München', 'München') res3 from (values(1));
++
|  res3  |
++
| false  |
++
1 row selected (0.148 seconds)
0: jdbc:drill:zk=local> select regexp_matches('abc', 'abc') from 
(values(1));
+-+
| EXPR$0  |
+-+
| true|
+-+
1 row selected (0.189 seconds)
0: jdbc:drill:zk=local> select 'München' = 'München' res1 from 
(values(1));
+---+
| res1  |
+---+
| true  |
+---+
1 row selected (0.186 seconds)
{code:sql}

Here is the result for 1st query, without your patch

{code:sql}
select regexp_matches('München', 'München') res3 from (values(1));
+---+
| res3  |
+---+
| true  |
+---+
{code:sql}

I think you should modify CharSequenceWrapper, so that the encoding method 
is honored.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4573 Fixed issue with regexp_replace fun...

2016-05-16 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/502#issuecomment-219577333
  
@jcmcote , I'll take a look at the patch shortly. You are right that the 
build issue should not be caused by your code change. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4514 : Add describe schema ...

2016-05-12 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/436#issuecomment-218841649
  
I'm not talking about implementation side 
(AbstractStoragePlugin.getPhysicalLocation etc). Before we go to 
implementation, we probably want to define what's the expected behavior for 
"describe schema".  For now, I'm not convinced that "physical_location" column 
in the output would make sense for non-filesystem schema.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4573: Zero copy LIKE, REGEXP_MATCHES, SU...

2016-05-12 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/458#issuecomment-218839576
  
@jcmcote 
I'll review your new patch if it's ready. 

Can you please submit the new patch as a new PR (or maybe I miss some part 
of conversation and just could not find the link)?

Thanks for your contribution to Drill. 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4514 : Add describe schema ...

2016-05-12 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/436#issuecomment-218786032
  
@arina-ielchiieva ,
1. the link you posted is actually not a public document. People in the 
community could not see the content. You may paste some discussion to the 
public JIRA, if you like.
2. I'm still not clear how you will populate "physical_location" for schema 
other than file system schema. For instance, HBase, MongoDb, JDBC. 
I understand that Hive will return "location". That makes sense, since Hive 
only deals with data on file systems. For Drill, it has to connect different 
storage plugin/schema, including file system schemas. If we put something only 
works for one type of storage plugin, then we will run into trouble in the 
future when we have to extend to other storage plugin.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4514 : Add describe schema ...

2016-05-11 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/436#issuecomment-218630539
  
The patch will show "schema_name, location" for "describe schema xxx" when 
it applied to file system schema. I'm not sure why you only want to make it 
work for file system schema, and return "information not available" for other 
schema. Also, what's the reason to only show "location"? What if we add support 
for other schemas, and will "location" still make sense for non-filesystem 
schema?

To me, it would make more sense to return all the properties of schema for 
"describe schema/database", as a JSON format string.  The properties could be 
anything defined in the schema. For file system schema, it could be location, 
whether writable, default input format, etc. For other schemas, there would be 
schema-dependent properties.  In case we add new properties to schema in later 
release, there is no need to change the implementation of "describe schema".  
In other words, "describe schema/database" should return the content shown in 
webui storage tab for that particular schema. 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4642: Remove customized RexBuilder.ensur...

2016-05-04 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/489#discussion_r62089298
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
 ---
@@ -719,4 +727,73 @@ public void testWindowSumConstant() throws Exception {
 final String[] excludedPlan = {};
 PlanTestBase.testPlanMatchingPatterns(query, expectedPlan, 
excludedPlan);
   }
+
+  @Test // DRILL-4552
+  public void testDecimalPlusWhenDecimalEnabled() throws Exception {
+final String query = "select cast('99' as decimal(9,0)) + cast('99' as 
decimal(9,0)) as col \n" +
+"from cp.`tpch/region.parquet` \n" +
+"limit 0";
+
+try {
+  final TypeProtos.MajorType majorTypeDouble = 
TypeProtos.MajorType.newBuilder()
+  .setMinorType(TypeProtos.MinorType.FLOAT8)
+  .setMode(TypeProtos.DataMode.REQUIRED)
+  .build();
+
+  final List<Pair<SchemaPath, TypeProtos.MajorType>> 
expectedSchemaDouble = Lists.newArrayList();
+  expectedSchemaDouble.add(Pair.of(SchemaPath.getSimplePath("col"), 
majorTypeDouble));
+
+  testBuilder()
--- End diff --

We had better to explicitly set the decimal option to false, before the 
test on line 746. For now, the option defaults to be false. But what if it's 
changed to true some day?




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4642: Remove customized RexBuilder.ensur...

2016-05-04 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/489#discussion_r62085424
  
--- Diff: 
exec/java-exec/src/test/java/org/apache/drill/TestFunctionsWithTypeExpoQueries.java
 ---
@@ -213,7 +221,7 @@ public void tesIsNull() throws Exception {
 List<Pair<SchemaPath, TypeProtos.MajorType>> expectedSchema = 
Lists.newArrayList();
 TypeProtos.MajorType majorType = TypeProtos.MajorType.newBuilder()
 .setMinorType(TypeProtos.MinorType.BIT)
-.setMode(TypeProtos.DataMode.REQUIRED)
+.setMode(TypeProtos.DataMode.OPTIONAL)
--- End diff --

why do you have to change the expected result for this query? Should  
"r_name is null" be a required type?  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4642: Remove customized RexBuilder.ensur...

2016-05-04 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/489#discussion_r62084962
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java
 ---
@@ -734,6 +772,41 @@ public static FunctionCall 
convertSqlOperatorBindingToFunctionCall(final SqlOper
   }
 
   /**
+   * Based on whether decimal type is supported or not, this method 
creates an ExplicitOperatorBinding which interprets
+   * the type of decimal literals accordingly.
+   */
+  public static SqlOperatorBinding convertDecimalLiteralToDouble(final 
SqlOperatorBinding sqlOperatorBinding, final boolean isDecimalSupported) {
+final List types = Lists.newArrayList();
+for(int i = 0; i < sqlOperatorBinding.getOperandCount(); ++i) {
+  final RelDataType relDataType;
+  if(isDecimalLiteral(sqlOperatorBinding, i) && !isDecimalSupported) {
+relDataType = createCalciteTypeWithNullability(
+sqlOperatorBinding.getTypeFactory(),
+SqlTypeName.DOUBLE,
+sqlOperatorBinding.getOperandType(i).isNullable());
+  } else {
+relDataType = sqlOperatorBinding.getOperandType(i);
+  }
+  types.add(relDataType);
+}
+return new 
ExplicitOperatorBinding(sqlOperatorBinding.getTypeFactory(), 
sqlOperatorBinding.getOperator(), types);
--- End diff --

Here we do not have to return a new OperatorBinding. In case there is no 
Decimal -> Double replacement, why can't we return the old binding directly? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4387: GroupScan or ScanBatchCreator shou...

2016-04-20 Thread jinfengni
Github user jinfengni closed the pull request at:

https://github.com/apache/drill/pull/379


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4577: Construct a specific path for quer...

2016-04-08 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/461#discussion_r59090757
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
 ---
@@ -72,4 +80,76 @@ public String getTypeName() {
 return HiveStoragePluginConfig.NAME;
   }
 
+  @Override
+  public List<Pair<String, ? extends Table>> getTablesByNames(final 
List tableNames) {
+final String schemaName = getName();
+final List<Pair<String, ? extends Table>> tableNameToTable = 
Lists.newArrayList();
+List tables;
+// Retries once if the first call to fetch the metadata fails
+synchronized(mClient) {
+  final List tableNamesWithAuth = Lists.newArrayList();
+  for(String tableName : tableNames) {
+try {
+  if(mClient.tableExists(schemaName, tableName)) {
--- End diff --

According to [1], under "Sql standard based authorization", Drill will 
return all the tables, even if the user does not have read access. That's the 
behavior before Sean's change to use bulk loading of getTableObjectsByNames(). 
However, under "Storage based authorization", the current expected behavior is 
only list the tables that user has access [2].

@vkorukanti , does this current behavior make sense? Why would Drill show 
different behavior under these two models?

Essentially, looks to me that the bulk loading will make Drill show same 
behavior under both "Sql standard based authorization", and "storage based 
authorization". That is, "show tables" will list all the tables, whether a user 
has access or not. But when a user query the table he does not have read 
access, then error will be raised.

[1] 
https://github.com/apache/drill/blob/master/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestSqlStdBasedAuthorization.java#L153

[2] 
https://github.com/apache/drill/blob/master/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/impersonation/hive/TestStorageBasedHiveAuthorization.java#L244-L247


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-07 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/468#issuecomment-207115616
  
@amansinha100 and @hsuanyi , I revised PR based on your comments. Can you 
take another look? thx.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/468#discussion_r58953148
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
 ---
@@ -0,0 +1,70 @@
+/**
+ * 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.
+ */
+
+/**
+ * Class defines a single partition corresponding to a directory in a DFS 
table.
+ */
+package org.apache.drill.exec.planner;
+
+
+import com.google.common.collect.Lists;
+
+import java.util.Collection;
+import java.util.List;
+
+public class DFSDirPartitionLocation implements PartitionLocation {
+  private final Collection subPartitions;
+  private final String[] dirs;
+
+  public DFSDirPartitionLocation(String[] dirs, 
Collection subPartitions) {
+this.subPartitions = subPartitions;
+this.dirs = dirs;
+  }
+
+  @Override
+  public String getPartitionValue(int index) {
+assert index < dirs.length;
+return dirs[index];
+  }
+
+  @Override
+  public String getEntirePartitionLocation() {
+throw new UnsupportedOperationException("Should not call 
getEntirePartitionLocation for composite partition location!");
+  }
+
+  @Override
+  public List getPartitionLocationRecursive() {
--- End diff --

I changed this method, such that now it returns list of 
SimplePartitionLocation.  This method would return all SimplePartitionLocation 
it consists of. In your example, it would return 4 DFSFilePartitionLocations, 
if it's called at the DFSDirPartitionLocation corresponding to '2016'. This 
method is used when we construct a GroupScan after pruning, since only 
SimplePartitionLocation keeps track the entire path, which is required by a 
groupscan specification.

The file partition location keeps track of full path (which would be used 
when created groupscan) and the partition keys. The dir keeps track the nested 
partition, and the common partition keys.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/468#discussion_r58949303
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
 ---
@@ -0,0 +1,70 @@
+/**
+ * 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.
+ */
+
+/**
+ * Class defines a single partition corresponding to a directory in a DFS 
table.
+ */
+package org.apache.drill.exec.planner;
+
+
+import com.google.common.collect.Lists;
+
+import java.util.Collection;
+import java.util.List;
+
+public class DFSDirPartitionLocation implements PartitionLocation {
+  private final Collection subPartitions;
--- End diff --

Yes, it could be mix of directory partition locations and file partition 
locations, similar to directory / file structures. 

Add comments to explain. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4592: Explain plan statement should show...

2016-04-07 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/471

DRILL-4592: Explain plan statement should show plan in WebUI

@amansinha100 , could you review this small code patch? Thanks. 

For now, there is lack of a way to add unit test for all WebUI related 
functionality. But I verified on my machine that the patch did show the plan in 
WebUI.



You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4592

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/471.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #471


commit 6345fd148ad7563841df93a18cf60e1ceab1464b
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-04-07T18:24:04Z

DRILL-4592: Explain plan statement should show plan in WebUI




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/468#discussion_r58912269
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSDirPartitionLocation.java
 ---
@@ -0,0 +1,70 @@
+/**
+ * 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.
+ */
+
+/**
+ * Class defines a single partition corresponding to a directory in a DFS 
table.
+ */
+package org.apache.drill.exec.planner;
+
+
+import com.google.common.collect.Lists;
+
+import java.util.Collection;
+import java.util.List;
+
+public class DFSDirPartitionLocation implements PartitionLocation {
+  private final Collection subPartitions;
+  private final String[] dirs;
+
+  public DFSDirPartitionLocation(String[] dirs, 
Collection subPartitions) {
+this.subPartitions = subPartitions;
+this.dirs = dirs;
+  }
+
+  @Override
+  public String getPartitionValue(int index) {
+assert index < dirs.length;
--- End diff --

this one actually is copied from [1]. I think it makes sense to change both 
to throw exception in stead of relying on assertion check. Will update the 
patch. 



https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/planner/DFSPartitionLocation.java#L58


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-07 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/468#discussion_r58911205
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/FileSystemPartitionDescriptor.java
 ---
@@ -148,13 +139,41 @@ public String getName(int index) {
 return partitionLabel + index;
   }
 
-  private String getBaseTableLocation() {
+  protected String getBaseTableLocation() {
--- End diff --

You are right.  it should remain as private. Originally, I intended to 
extend this class. But I decided to remove that part of code from this PR. Will 
update the patch. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-06 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/468#issuecomment-206611168
  
@amansinha100 , could you please review this PR? thanks!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4589: Reduce planning time for file syst...

2016-04-06 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/468

DRILL-4589: Reduce planning time for file system partition pruning by…

… reducing filter evaluation overhead

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4589

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/468.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #468


commit e207a926e65cd788700229de3ae47cf4e876
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-02-25T18:13:43Z

DRILL-4589: Reduce planning time for file system partition pruning by 
reducing filter evaluation overhead




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4577: Construct a specific path for quer...

2016-04-05 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/461#issuecomment-205991638
  
@hsuanyi , did you do some preliminary performance comparison, to measure 
the performance gain, with bulk loading and skip get_partition() for hive table 
when doing infor schema query? If you have some results, please post in 
DRILL-4577. 
 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4577: Construct a specific path for quer...

2016-04-05 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/461#discussion_r58616431
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java ---
@@ -194,4 +195,26 @@ public void dropTable(String tableName) {
 .message("Dropping tables is not supported in schema [%s]", 
getSchemaPath())
 .build(logger);
   }
-}
+
+  /**
+   * Visit the tables in this schema and write to recordGenerator
+   * @param recordGenerator recordGenerator for the output
+   * @param schemaPath  the path to the given schema
+   */
--- End diff --

Why do you add this API to AbstractSchema? We should not put the logic of 
RecordGenerator in Schema. 

A cleaner way is to add getTablesByNames() in AbstractSchema, and leave the 
logic of iterating and record generating to RecordGenerator.  If the underlying 
storage supports bulk load ( like Hive does), then use the bulk load. 
Otherwise, just call getTable() individually for getTablesByNames(). 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4551: Implement new functions (cot, rege...

2016-03-30 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/452#issuecomment-203744127
  
Other than one comment, look good to me.

+1




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4551: Implement new functions (cot, rege...

2016-03-30 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/452#discussion_r57998932
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
 ---
@@ -40,6 +42,41 @@
 
 public class DateTypeFunctions {
 
+/**
+ * Function to check if a varchar value can be cast to a date.
+ *
+ * At the time of writing this function, several other databases were 
checked
+ * for behavior compatibility. There was not a consensus between 
oracle and
+ * Sql server about the expected behavior of this function, and 
Postgres
+ * lacks it completely.
+ *
+ * Sql Server appears to have both a DATEFORMAT and language locale 
setting
+ * that can change the values accepted by this function. Oracle 
appears to
+ * support several formats, some of which are not mentioned in the Sql
+ * Server docs. With the lack of standardization, we decided to 
implement
+ * this function so that it would only consider date strings that 
would be
+ * accepted by the cast function as valid.
+ */
+@SuppressWarnings("unused")
+@FunctionTemplate(name = "isdate", scope = 
FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.INTERNAL,
+costCategory = FunctionTemplate.FunctionCostCategory.COMPLEX)
+public static class CastVarCharToDate implements DrillSimpleFunc {
+
+  @Param NullableVarCharHolder in;
--- End diff --

Do we need an implementation for non-nullable input, since it's using 
NullHandling.INTERNAL? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4529: Force $SUM0 to be used when Window...

2016-03-30 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/447#issuecomment-203461091
  
Since we have use same approach for regular sum() aggregate, it makes sense 
to apply to window sum() aggregate function.

LGTM

+1. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4551: Implement new functions (cot, rege...

2016-03-29 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/452#discussion_r57819554
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctionHelpers.java
 ---
@@ -213,11 +213,39 @@ public static long getDate(DrillBuf buf, int start, 
int end){
 if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
   buf.checkBytes(start, end);
 }
-return memGetDate(buf.memoryAddress(), start, end);
+int[] dateFields = memGetDate(buf.memoryAddress(), start, end);
+return CHRONOLOGY.getDateTimeMillis(dateFields[0], dateFields[1], 
dateFields[2], 0);
   }
 
+  /**
+   * Takes a string value, specified as a buffer with a start and end and
+   * returns true if the value can be read as a date.
+   *
+   * @param buf
+   * @param start
+   * @param end
+   * @return true iff the string value can be read as a date
+   */
+  public static boolean isReadableAsDate(DrillBuf buf, int start, int end){
+if (BoundsChecking.BOUNDS_CHECKING_ENABLED) {
+  buf.checkBytes(start, end);
+}
+int[] dateFields = memGetDate(buf.memoryAddress(), start, end);
--- End diff --

Can we call getDate() directly here, and wrap with a try/catch block? The 
code seems identical to getDate(), except for the try/catch block. 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4551: Implement new functions (cot, rege...

2016-03-29 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/452#discussion_r57818552
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
 ---
@@ -40,6 +41,36 @@
 
 public class DateTypeFunctions {
 
+/**
+ * Function to check if a varchar value can be cast to a date.
+ *
+ * At the time of writing this function, several other databases were 
checked
+ * for behavior compatibility. There was not a consensus between 
oracle and
+ * Sql server about the expected behavior of this function, and 
Postgres
+ * lacks it completely.
+ *
+ * Sql Server appears to have both a DATEFORMAT and language locale 
setting
+ * that can change the values accepted by this function. Oracle 
appears to
+ * support several formats, some of which are not mentioned in the Sql
+ * Server docs. With the lack of standardization, we decided to 
implement
+ * this function so that it would only consider date strings that 
would be
+ * accepted by the cast function as valid.
+ */
+@SuppressWarnings("unused")
+@FunctionTemplate(name = "isdate", scope = 
FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL,
--- End diff --

Have you checked isdate() returns null for null input in other system like 
oracle? I thought it would return either true or false. 
 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4531: Add a Drill customized rule for pu...

2016-03-25 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/444#issuecomment-201430413
  
I agree that your expectation for RelSubset makes sense to me. However, for 
now it does not happen that way. The following is the trace for the query which 
went through planning with this customized rule (I removed some rels).

Set#1, type: RecordType(BIGINT custkey, ANY custAddress)
...
Set#2, type: (DrillRecordRow[*, l_orderkey, l_partkey, l_linenumber])
  rel#273:Subset#2.LOGICAL.ANY([]).[], best=rel#442, 
importance=0.3138105960906

rel#275:AbstractConverter.LOGICAL.ANY([]).[](input=rel#80:Subset#2.ENUMERABLE.ANY([]).[],convention=LOGICAL,DrillDistributionTraitDef=ANY([]),sort=[]),
 rowcount=100.0, cumulative cost={inf}
rel#442:DrillScanRel.LOGICAL.ANY([]).[](table=[cp, 
tpch/lineitem.parquet],groupscan=ParquetGroupScan [entries=[ReadEntryWithPath 
[path=classpath:/tpch/lineitem.parquet]], 
selectionRoot=classpath:/tpch/lineitem.parquet, numFiles=1, 
usedMetadataFile=false, columns=[`*`]]), rowcount=60175.0, cumulative 
cost={60175.0 rows, 6.0175E8 cpu, 0.0 io, 0.0 network, 0.0 memory}
Set#3, type: (DrillRecordRow[*, l_orderkey, l_partkey, l_linenumber])
  rel#82:Subset#3.NONE.ANY([]).[], best=null, importance=0.387420489001

rel#345:LogicalFilter.NONE.ANY([]).[[]](input=rel#273:Subset#2.LOGICAL.ANY([]).[],condition=AND(>=($1,
 20160101), <=($2, 20160301), OR(=($2, 1), =($2, 2), =($2, 5), =($2, 6, 
rowcount=6.25, cumulative cost={inf}

rel#345:LogicalFilter has a child rel#273 with LOGICAL convention.

As another example, for the following query:
 
   Select n_name, n_nationkey from cp.`tpch/nation.parquet` where 
n_nationkey > 5

The trace:
Set#0, type: (DrillRecordRow[*, n_nationkey, n_name])
  ... 
Set#1, type: (DrillRecordRow[*, n_nationkey, n_name])
  rel#21:Subset#1.NONE.ANY([]).[], best=null, importance=0.81

rel#20:LogicalFilter.NONE.ANY([]).[](input=rel#19:Subset#0.ENUMERABLE.ANY([]).[],condition=>($1,
 5)), rowcount=50.0, cumulative cost={inf}

rel#37:LogicalFilter.NONE.ANY([]).[[]](input=rel#19:Subset#0.ENUMERABLE.ANY([]).[],condition=>($1,
 5)), rowcount=50.0, cumulative cost={inf}
  rel#35:Subset#1.LOGICAL.ANY([]).[], best=rel#60, importance=0.81

rel#60:DrillFilterRel.LOGICAL.ANY([]).[](input=rel#57:Subset#0.LOGICAL.ANY([]).[],condition=>($1,
 5)), rowcount=50.0, cumulative cost={125.0 rows, 250600.0 cpu, 0.0 io, 0.0 
network, 0.0 memory}
Set#2, type: RecordType(ANY n_name, ANY n_nationkey)
  rel#23:Subset#2.NONE.ANY([]).[], best=null, importance=0.9

rel#53:LogicalProject.NONE.ANY([]).[](input=rel#35:Subset#1.LOGICAL.ANY([]).[],n_name=$2,n_nationkey=$1),
 rowcount=50.0, cumulative cost={inf}

Again, rel#53:LogicalProject has a child rel#35 whose convention is LOGICAL.

I think the reason that we have such mixed rels is we have different kinds 
of rules, used in a single Volcano planning phase.
 1) Rule matchs base class Filter/Project, etc only.
 2) Rule matches LogicalFilter/LogicalProject, etc
 3) Rule uses copy() method to generate a new Rel 
 4) Rule  uses RelFactory to generate a new Rel.
 5) convent rule, which convert from Calcite logical (NONE/Enumerable) to 
Drill logical (LOGICAL)

For instance, ProjectMergeRule, which matches base Project, yet uses 
default RelFactory, will match both LogicalProject and DrillProject, but 
produce LogicalProject as outcome. That will cause the mixed rels. 

2 things we may consider to fix this:
1) Separate the convent rules from the other transformation rules. Apply 
convert rule first, then transformation rule match DrillLogical only. That's 
similar to what other system (hive) is doing.
2) go through every rule we use, and we need make sure the convention of 
input and ouptput of a transformation rule should be same, except for the 
convert rule.

The above 2 things would take some considerably effort, though.

  



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4531: Add a Drill customized rule for pu...

2016-03-24 Thread jinfengni
GitHub user jinfengni opened a pull request:

https://github.com/apache/drill/pull/444

DRILL-4531: Add a Drill customized rule for pushing filter past aggre…

…gate

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/jinfengni/incubator-drill DRILL-4531

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/drill/pull/444.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #444


commit c7ee169058f8e53a2796a0986a15fe2994aeab8e
Author: Jinfeng Ni <j...@apache.org>
Date:   2016-03-23T00:57:02Z

DRILL-4531: Add a Drill customized rule for pushing filter past aggregate




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4525: Allow SqlBetweenOperator to accept...

2016-03-24 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/439#issuecomment-200907542
  
My understanding is adding another operator will not work, since Calcite 
parser has put the built-in between operator in the sql expression tree[1]. The 
validator logic will simply use the build-in operator, even we add another 
operator. 

[1] 
https://github.com/apache/calcite/blob/master/core/src/main/codegen/templates/Parser.jj#L2564


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-4525: Allow SqlBetweenOperator to accept...

2016-03-24 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/439#issuecomment-200883747
  
Looks like the root cause of the problem is Calcite does not allow the 
comparison of Date and Timestamp.

{code}
select CAST('1990-01-01' AS DATE) < CAST('2001-01-01' AS TIMESTAMP) FROM 
(VALUES(1, 2)) AS T(A,B);
Mar 24, 2016 8:15:53 AM 
org.apache.calcite.sql.validate.SqlValidatorException 
SEVERE: org.apache.calcite.sql.validate.SqlValidatorException: Cannot apply 
'<' to arguments of type ' < <TIMESTAMP(0)>'. Supported form(s): 
' < '
{code}

Postgres, on the country, supports date vs timestamp compare, through 
implicit cast.

{code}
select CAST('1990-01-01' AS DATE) < CAST('2001-01-01' AS TIMESTAMP) FROM 
(VALUES(1, 2)) AS T(A,B);
 ?column?
--
 t
(1 row)
{code}

I looked through SQL standard, and could not find the rules for the 
implicit cast, which means different system could choose it's own 
interpretation.  (In other words, it's hard to argue what Calcite is doing is 
not right)

What Sean is trying to do with his proposal is to replace Calcite's between 
operator with Drill's between operator, so that the checkOperand will use 
Drill's rule.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: DRILL-3623: For limit 0 queries, use a shorter...

2016-03-22 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/405#issuecomment-20859
  
LGTM.

+1 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56426515
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java
 ---
@@ -122,6 +127,13 @@ public int size(){
   }
 
   public void register(DrillOperatorTable operatorTable) {
+registerForInference(operatorTable);
+registerForDefault(operatorTable);
--- End diff --

"Default" is not a good name (You choose "default", just because that's the 
existing behavior). Better to change to more meaningful name. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-20 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56427623
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java
 ---
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+
+import java.util.ArrayList;
+
+public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
+  private static final TypeProtos.MajorType NONE = 
TypeProtos.MajorType.getDefaultInstance();
+  private final TypeProtos.MajorType returnType;
+
+  public DrillSqlOperatorNotInfer(String name, int argCount, 
TypeProtos.MajorType returnType, boolean isDeterminisitic) {
+super(name,
+new ArrayList< DrillFuncHolder>(),
+argCount,
+argCount,
+isDeterminisitic,
+DynamicReturnType.INSTANCE);
+this.returnType = Preconditions.checkNotNull(returnType);
+  }
+
+  protected RelDataType getReturnDataType(final RelDataTypeFactory 
factory) {
+if (TypeProtos.MinorType.BIT.equals(returnType.getMinorType())) {
+  return factory.createSqlType(SqlTypeName.BOOLEAN);
+}
+return 
factory.createTypeWithNullability(factory.createSqlType(SqlTypeName.ANY), true);
+  }
+
+  private RelDataType getNullableReturnDataType(final RelDataTypeFactory 
factory) {
+return factory.createTypeWithNullability(getReturnDataType(factory), 
true);
+  }
+
+  @Override
+  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope 
scope, SqlCall call) {
+if (NONE.equals(returnType)) {
+  return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
--- End diff --

Nullability of "any" type? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56435067
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
 ---
@@ -64,4 +108,47 @@ public boolean isDeterministic() {
   public List getFunctions() {
 return functions;
   }
+
+  public static class DrillSqlOperatorBuilder {
+private String name;
+private final List functions = Lists.newArrayList();
+private int argCountMin = Integer.MAX_VALUE;
+private int argCountMax = Integer.MIN_VALUE;
+private boolean isDeterministic = true;
+
+public DrillSqlOperatorBuilder setName(final String name) {
+  this.name = name;
+  return this;
+}
+
+public DrillSqlOperatorBuilder 
addFunctions(Collection functions) {
+  this.functions.addAll(functions);
+  return this;
+}
+
+public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, 
final int argCountMax) {
+  this.argCountMin = Math.min(this.argCountMin, argCountMin);
+  this.argCountMax = Math.max(this.argCountMax, argCountMax);
+  return this;
+}
+
+public DrillSqlOperatorBuilder setDeterministic(boolean 
isDeterministic) {
+  if(this.isDeterministic) {
--- End diff --

why do you have this check?

If someone uses this builder, setDeterminsitic(false), then 
setDeterministic(true). The second call will not get the desired result.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on the pull request:

https://github.com/apache/drill/pull/397#issuecomment-198407062
  
Looks good to me. 

+1

Some additional info

1. This PR has added an option to enable/disable the type inference feature 
(by default, it's enabled). 
2. Internal performance shows this type inference feature, combined with 
"limit 0" optimization,  shows huge improvement for "limit 0" query, some time 
with orders of magnitude. 
3. It has passed the pre-commit test with option enabled/disabled.
4. It has addressed issues found by MapR QA folks during their extensive 
testing.
5. @hsuanyi will add a doc to the JIRA, showing the scope where the type 
inference feature would or would not be applied.  
6. As a side note, this PR requires one patch in Drill forked Calcite. The 
change is needed because of the star column changes specifically made for Drill 
in forked Calcite.  For now, it's not feasible to push this change to Calcite 
master, until the rebasing work is done.

@jacques-n , @amansinha100 , @sudheeshkatkam , if you guys have any comment 
or concern, please let @hsuanyi know. If no concern, I'm going to merge this PR 
to master branch. 
  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56524249
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
 ---
@@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus 
schema) {
 }
   }
 
+  private static class DrillRexBuilder extends RexBuilder {
+private DrillRexBuilder(RelDataTypeFactory typeFactory) {
+  super(typeFactory);
+}
+
+@Override
+public RexNode ensureType(
+RelDataType type,
+RexNode node,
+boolean matchNullability) {
+  RelDataType targetType = type;
+  if (matchNullability) {
+targetType = matchNullability(type, node);
+  }
+  if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
+return node;
+  }
+  if (!node.getType().equals(targetType)) {
+if(!targetType.isStruct()) {
+  final RelDataType anyType = 
TypeInferenceUtils.createCalciteTypeWithNullability(
--- End diff --

Cast an expression to "any" type? why do we have to add this? On line 418, 
in Calcite, the code's intention is to skip CAST to "any", since it does not 
make sense to cast to "any". Why do you want to explicitly add this here?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56458894
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
 ---
@@ -64,4 +108,47 @@ public boolean isDeterministic() {
   public List getFunctions() {
 return functions;
   }
+
+  public static class DrillSqlOperatorBuilder {
+private String name;
+private final List functions = Lists.newArrayList();
+private int argCountMin = Integer.MAX_VALUE;
+private int argCountMax = Integer.MIN_VALUE;
+private boolean isDeterministic = true;
+
+public DrillSqlOperatorBuilder setName(final String name) {
+  this.name = name;
+  return this;
+}
+
+public DrillSqlOperatorBuilder 
addFunctions(Collection functions) {
+  this.functions.addAll(functions);
+  return this;
+}
+
+public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, 
final int argCountMax) {
+  this.argCountMin = Math.min(this.argCountMin, argCountMin);
+  this.argCountMax = Math.max(this.argCountMax, argCountMax);
+  return this;
+}
+
+public DrillSqlOperatorBuilder setDeterministic(boolean 
isDeterministic) {
+  if(this.isDeterministic) {
--- End diff --

In that case, pls add some comments here. Otherwise, the code is kind of 
confusing; people would have hard time to understand later on. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56427768
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java
 ---
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+
+import java.util.ArrayList;
+
+public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
--- End diff --

pls consider using a different class name. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-19 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56432884
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java
 ---
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.planner.sql;
+
+import com.google.common.base.Preconditions;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+
+import java.util.ArrayList;
+
+public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
+  private static final TypeProtos.MajorType NONE = 
TypeProtos.MajorType.getDefaultInstance();
+  private final TypeProtos.MajorType returnType;
+
+  public DrillSqlOperatorNotInfer(String name, int argCount, 
TypeProtos.MajorType returnType, boolean isDeterminisitic) {
+super(name,
+new ArrayList< DrillFuncHolder>(),
+argCount,
+argCount,
+isDeterminisitic,
+DynamicReturnType.INSTANCE);
--- End diff --

This is not right. You pass DynamicReturnType.INSTANCE as 
SqlReturnTypeInference, yet also override inferReturnType. Why do you have to 
pass DynamicReturnType here? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-18 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56426363
  
--- Diff: 
contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperatorNotInfer.java
 ---
@@ -0,0 +1,44 @@
+/**
+ * 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.drill.exec.planner.sql;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+
+public class HiveUDFOperatorNotInfer extends HiveUDFOperator {
--- End diff --

The class name is not very meaningful. Can we use other names to different 
the concept of type inference? 




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] drill pull request: Drill 4372 review

2016-03-18 Thread jinfengni
Github user jinfengni commented on a diff in the pull request:

https://github.com/apache/drill/pull/397#discussion_r56439767
  
--- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
 ---
@@ -183,10 +190,40 @@ public SchemaPlus getDefaultSchema() {
   }
 
   private class DrillValidator extends SqlValidatorImpl {
+private final Set identitySet = 
Sets.newIdentityHashSet();
+
 protected DrillValidator(SqlOperatorTable opTab, 
SqlValidatorCatalogReader catalogReader,
 RelDataTypeFactory typeFactory, SqlConformance conformance) {
   super(opTab, catalogReader, typeFactory, conformance);
 }
+
+@Override
+public SqlValidatorScope getSelectScope(final SqlSelect select) {
--- End diff --

We need better solution here. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


<    1   2   3   4   5   6   >