[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201714214
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestComplexColumnInSchema.java
 ##
 @@ -0,0 +1,141 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+
+/**
+ * This test checks correctness of complex column detection in the Parquet 
file schema.
+ */
+public class TestComplexColumnInSchema {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex_special_cases.parquet";
+  private static ParquetMetadata footer;
+  List columns;
 
 Review comment:
   This and all other outstanding comments have been addressed.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201672191
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderDecision.java
 ##
 @@ -0,0 +1,99 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+/**
+ * DRILL-5797 introduces more granularity on new reader use cases. This test 
is aimed at
+ * checking correctness of function used for new reader usage decision making.
+ */
+public class TestParquetReaderDecision {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
+  + " cwd: " + Paths.get(".").toAbsolutePath().normalize().toString());
+  throw(ioe);
+}
+  }
+
+  @Test
+  public void testParquetReaderDecision() {
 
 Review comment:
   Done, schema is:
   ```
   file schema:   root
   

   id:OPTIONAL INT64 R:0 D:1
   a: OPTIONAL BINARY O:UTF8 R:0 D:1
   repeated:  REPEATED INT64 R:1 D:1
   VariableCase:  OPTIONAL BINARY O:UTF8 R:0 D:1
   nested:OPTIONAL F:3
   .id:   OPTIONAL INT64 R:0 D:2
   .repeated: REPEATED INT64 R:1 D:2
   .VaRiAbLeCaSe: OPTIONAL BINARY O:UTF8 R:0 D:2
   ```


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663784
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -191,4 +201,42 @@ protected AbstractDrillFileSystemManager(OperatorContext 
operatorContext) {
 protected abstract DrillFileSystem get(Configuration config, String path) 
throws ExecutionSetupException;
   }
 
+  /**
+   * Check whether any of queried columns is nested or repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List columns) {
 
 Review comment:
   Method moved to ParquetReaderUtility as per discussion below.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663521
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
 ##
 @@ -140,15 +140,87 @@ public static int getIntFromLEBytes(byte[] input, int 
start) {
 return out;
   }
 
+  /**
+   * Map full schema paths in format `a`.`b`.`c` to respective SchemaElement 
objects.
+   *
+   * @param footer Parquet file metadata
+   * @return   schema full path to SchemaElement map
+   */
   public static Map 
getColNameToSchemaElementMapping(ParquetMetadata footer) {
-HashMap schemaElements = new HashMap<>();
+Map schemaElements = new HashMap<>();
 FileMetaData fileMetaData = new 
ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION, 
footer);
-for (SchemaElement se : fileMetaData.getSchema()) {
-  schemaElements.put(se.getName(), se);
+
+Iterator iter = fileMetaData.getSchema().iterator();
+
+// skip first default 'root' element
+if (iter.hasNext()) {
 
 Review comment:
   Done


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663590
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetReaderUtility.java
 ##
 @@ -140,15 +140,87 @@ public static int getIntFromLEBytes(byte[] input, int 
start) {
 return out;
   }
 
+  /**
+   * Map full schema paths in format `a`.`b`.`c` to respective SchemaElement 
objects.
+   *
+   * @param footer Parquet file metadata
+   * @return   schema full path to SchemaElement map
+   */
   public static Map 
getColNameToSchemaElementMapping(ParquetMetadata footer) {
-HashMap schemaElements = new HashMap<>();
+Map schemaElements = new HashMap<>();
 FileMetaData fileMetaData = new 
ParquetMetadataConverter().toParquetMetadata(ParquetFileWriter.CURRENT_VERSION, 
footer);
-for (SchemaElement se : fileMetaData.getSchema()) {
-  schemaElements.put(se.getName(), se);
+
+Iterator iter = fileMetaData.getSchema().iterator();
+
+// skip first default 'root' element
+if (iter.hasNext()) {
+  iter.next();
+}
+while (iter.hasNext()) {
+  addSchemaElementMapping(iter, new StringBuilder(), schemaElements);
 }
 return schemaElements;
   }
 
+  /**
+   * Populate full path to SchemaElement map by recursively traversing schema 
elements referenced by the given iterator
+   *
+   * @param iter file schema values iterator
+   * @param path parent schema element path
+   * @param schemaElements schema elements map to insert next iterator element 
into
+   */
+  private static void addSchemaElementMapping(Iterator iter, StringBuilder 
path,
+  Map schemaElements) {
+SchemaElement se = (SchemaElement)iter.next();
 
 Review comment:
   Fixed both


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663491
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderUtility.java
 ##
 @@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.format.SchemaElement;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Map;
+
+public class TestParquetReaderUtility {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
 
 Review comment:
   Exception handling removed as per above comment.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663214
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderUtility.java
 ##
 @@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.format.SchemaElement;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Map;
+
+public class TestParquetReaderUtility {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
+  + " cwd: " + Paths.get(".").toAbsolutePath().normalize().toString());
+  throw(ioe);
+}
+  }
+
+  @Test
+  public void testSchemaElementsMap() {
+assertTrue(footer != null);
 
 Review comment:
   Removed the check and let failure to be handled via throwing the exception 
from setup method.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663328
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderUtility.java
 ##
 @@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.format.SchemaElement;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Map;
+
+public class TestParquetReaderUtility {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
 
 Review comment:
   Fixed


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201663246
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderUtility.java
 ##
 @@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import org.apache.drill.exec.store.parquet.ParquetReaderUtility;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.format.SchemaElement;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+import java.util.Map;
+
+public class TestParquetReaderUtility {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
+  + " cwd: " + Paths.get(".").toAbsolutePath().normalize().toString());
+  throw(ioe);
+}
+  }
+
+  @Test
+  public void testSchemaElementsMap() {
+assertTrue(footer != null);
+Map schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+assertTrue(schemaElements.size() == 14);
+
+SchemaElement se = schemaElements.get("`marketing_info`.`camp_id`");
+assertTrue(se != null);
+assertTrue("camp_id".equals(se.getName()));
 
 Review comment:
   Done


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201662880
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderDecision.java
 ##
 @@ -0,0 +1,99 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+/**
+ * DRILL-5797 introduces more granularity on new reader use cases. This test 
is aimed at
+ * checking correctness of function used for new reader usage decision making.
+ */
+public class TestParquetReaderDecision {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
+  + " cwd: " + Paths.get(".").toAbsolutePath().normalize().toString());
+  throw(ioe);
+}
+  }
+
+  @Test
+  public void testParquetReaderDecision() {
+List caseOldReader1 = new ArrayList<>();
+List caseOldReader2 = new ArrayList<>();
+List caseOldReader3 = new ArrayList<>();
+List caseNewReader1 = new ArrayList<>();
+List caseNewReader2 = new ArrayList<>();
+List caseNewReader3 = new ArrayList<>();
+
+SchemaPath topNestedPath = SchemaPath.getCompoundPath("marketing_info");
+SchemaPath nestedColumnPath = SchemaPath.getCompoundPath("marketing_info", 
"camp_id");
+SchemaPath topPath1 = SchemaPath.getCompoundPath("trans_id");
+SchemaPath topPath2 = SchemaPath.getCompoundPath("amount");
+SchemaPath nonExistentPath = SchemaPath.getCompoundPath("nonexistent");
+
+caseOldReader1.add(topNestedPath);
+caseOldReader2.add(nestedColumnPath);
+caseOldReader3.add(topPath1);
+caseOldReader3.add(nestedColumnPath);
+
+caseNewReader1.add(topPath1);
+caseNewReader2.add(topPath1);
+caseNewReader2.add(topPath2);
+
+assertTrue("Complex column not detected",
+AbstractParquetScanBatchCreator.containsComplexColumn(footer, 
caseOldReader1));
+assertTrue("Complex column not detected",
+AbstractParquetScanBatchCreator.containsComplexColumn(footer, 
caseOldReader2));
+assertTrue("Complex column not detected",
+AbstractParquetScanBatchCreator.containsComplexColumn(footer, 
caseOldReader3));
+
+assertFalse("No complex column, but complex column is detected",
+AbstractParquetScanBatchCreator.containsComplexColumn(footer, 
caseNewReader1));
 
 Review comment:
   `containsComplexColumn` method moved to ParquetReaderUtility class.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201662542
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderDecision.java
 ##
 @@ -0,0 +1,99 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+/**
+ * DRILL-5797 introduces more granularity on new reader use cases. This test 
is aimed at
+ * checking correctness of function used for new reader usage decision making.
+ */
+public class TestParquetReaderDecision {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
 
 Review comment:
   Removed exception handling as failure reason would indeed be obvious.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201662352
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderDecision.java
 ##
 @@ -0,0 +1,99 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+/**
+ * DRILL-5797 introduces more granularity on new reader use cases. This test 
is aimed at
+ * checking correctness of function used for new reader usage decision making.
+ */
+public class TestParquetReaderDecision {
 
 Review comment:
   Class renamed to `TestComplexColumnInSchema`


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201661923
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -191,4 +201,42 @@ protected AbstractDrillFileSystemManager(OperatorContext 
operatorContext) {
 protected abstract DrillFileSystem get(Configuration config, String path) 
throws ExecutionSetupException;
   }
 
+  /**
+   * Check whether any of queried columns is nested or repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List columns) {
+
+Map colDescMap = 
ParquetReaderUtility.getColNameToColumnDescriptorMapping(footer);
+Map schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+
+for (SchemaPath schemaPath : columns) {
+  // non-nested column check: full path must be equal to root segment path
+  if (!schemaPath.getUnIndexed().toString().replaceAll("`", "")
+  .equalsIgnoreCase(schemaPath.getRootSegment().getPath())) {
+logger.debug("Forcing 'old' reader due to nested column: {}", 
schemaPath.getUnIndexed().toString());
+return true;
+  }
+
+  // following column descriptor lookup failure may mean two cases, 
depending on subsequent SchemaElement lookup:
+  // 1. success: queried column is complex => use old reader
+  // 2. failure: queried column is not in schema => use new reader
+  ColumnDescriptor column = 
colDescMap.get(schemaPath.getUnIndexed().toString().toLowerCase());
+
+  if (column == null) {
+SchemaElement se = 
schemaElements.get(schemaPath.getUnIndexed().toString().toLowerCase());
 
 Review comment:
   Fixed


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201661724
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -161,20 +167,24 @@ private ParquetMetadata readFooter(Configuration conf, 
String path) throws IOExc
 }
   }
 
-  private boolean isComplex(ParquetMetadata footer) {
+  private boolean isComplex(ParquetMetadata footer, List columns) {
 MessageType schema = footer.getFileMetaData().getSchema();
 
-for (Type type : schema.getFields()) {
-  if (!type.isPrimitive()) {
-return true;
+if (Utilities.isStarQuery(columns)) {
+  for (Type type : schema.getFields()) {
+if (!type.isPrimitive()) {
+  return true;
+}
   }
-}
-for (ColumnDescriptor col : schema.getColumns()) {
-  if (col.getMaxRepetitionLevel() > 0) {
-return true;
+  for (ColumnDescriptor col : schema.getColumns()) {
+if (col.getMaxRepetitionLevel() > 0) {
+  return true;
+}
   }
+  return false;
+} else {
+  return containsComplexColumn(footer, columns);
 
 Review comment:
   Methods combined in one and moved to utility class.


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201661782
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -191,4 +201,42 @@ protected AbstractDrillFileSystemManager(OperatorContext 
operatorContext) {
 protected abstract DrillFileSystem get(Configuration config, String path) 
throws ExecutionSetupException;
   }
 
+  /**
+   * Check whether any of queried columns is nested or repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List columns) {
+
+Map colDescMap = 
ParquetReaderUtility.getColNameToColumnDescriptorMapping(footer);
+Map schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+
+for (SchemaPath schemaPath : columns) {
+  // non-nested column check: full path must be equal to root segment path
+  if (!schemaPath.getUnIndexed().toString().replaceAll("`", "")
+  .equalsIgnoreCase(schemaPath.getRootSegment().getPath())) {
+logger.debug("Forcing 'old' reader due to nested column: {}", 
schemaPath.getUnIndexed().toString());
+return true;
+  }
+
+  // following column descriptor lookup failure may mean two cases, 
depending on subsequent SchemaElement lookup:
+  // 1. success: queried column is complex => use old reader
+  // 2. failure: queried column is not in schema => use new reader
+  ColumnDescriptor column = 
colDescMap.get(schemaPath.getUnIndexed().toString().toLowerCase());
+
+  if (column == null) {
+SchemaElement se = 
schemaElements.get(schemaPath.getUnIndexed().toString().toLowerCase());
+if (se != null) {
+  return true;
+}
+  } else {
+if (column.getMaxRepetitionLevel() > 0) {
+  logger.debug("Forcing 'old' reader due to repetitive column: {}", 
schemaPath.getUnIndexed().toString());
 
 Review comment:
   Fixed


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201661554
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -191,4 +201,42 @@ protected AbstractDrillFileSystemManager(OperatorContext 
operatorContext) {
 protected abstract DrillFileSystem get(Configuration config, String path) 
throws ExecutionSetupException;
   }
 
+  /**
+   * Check whether any of queried columns is nested or repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List columns) {
+
+Map colDescMap = 
ParquetReaderUtility.getColNameToColumnDescriptorMapping(footer);
+Map schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+
+for (SchemaPath schemaPath : columns) {
+  // non-nested column check: full path must be equal to root segment path
+  if (!schemaPath.getUnIndexed().toString().replaceAll("`", "")
+  .equalsIgnoreCase(schemaPath.getRootSegment().getPath())) {
+logger.debug("Forcing 'old' reader due to nested column: {}", 
schemaPath.getUnIndexed().toString());
 
 Review comment:
   Fixed


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-11 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201661188
 
 

 ##
 File path: 
exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/AbstractParquetScanBatchCreator.java
 ##
 @@ -191,4 +201,42 @@ protected AbstractDrillFileSystemManager(OperatorContext 
operatorContext) {
 protected abstract DrillFileSystem get(Configuration config, String path) 
throws ExecutionSetupException;
   }
 
+  /**
+   * Check whether any of queried columns is nested or repetitive.
+   *
+   * @param footer  Parquet file schema
+   * @param columns list of query SchemaPath objects
+   */
+  public static boolean containsComplexColumn(ParquetMetadata footer, 
List columns) {
+
+Map colDescMap = 
ParquetReaderUtility.getColNameToColumnDescriptorMapping(footer);
+Map schemaElements = 
ParquetReaderUtility.getColNameToSchemaElementMapping(footer);
+
+for (SchemaPath schemaPath : columns) {
+  // non-nested column check: full path must be equal to root segment path
+  if (!schemaPath.getUnIndexed().toString().replaceAll("`", "")
 
 Review comment:
   Replaced with `SchemaPath.isLeaf()`


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


With regards,
Apache Git Services


[GitHub] okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet new reader in all non-complex column queries

2018-07-10 Thread GitBox
okalinin commented on a change in pull request #1370: DRILL-5797: Use Parquet 
new reader in all non-complex column queries
URL: https://github.com/apache/drill/pull/1370#discussion_r201517913
 
 

 ##
 File path: 
exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetReaderDecision.java
 ##
 @@ -0,0 +1,99 @@
+/*
+ * 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.
+ */
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.store.parquet.AbstractParquetScanBatchCreator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.ParquetFileReader;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.ArrayList;
+
+import java.io.IOException;
+import java.nio.file.Paths;
+
+/**
+ * DRILL-5797 introduces more granularity on new reader use cases. This test 
is aimed at
+ * checking correctness of function used for new reader usage decision making.
+ */
+public class TestParquetReaderDecision {
+
+  private static final String path = 
"src/test/resources/store/parquet/complex/complex.parquet";
+  private static Configuration conf;
+  private static ParquetMetadata footer;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+conf = new Configuration();
+
+try {
+  footer = ParquetFileReader.readFooter(conf, new Path(path));
+} catch (IOException ioe) {
+  fail("Could not read Parquet file '" + path + "', error message: " + 
ioe.getMessage()
+  + " cwd: " + Paths.get(".").toAbsolutePath().normalize().toString());
+  throw(ioe);
+}
+  }
+
+  @Test
+  public void testParquetReaderDecision() {
 
 Review comment:
   I will fix this and all other test issues you highlighted. When fixing them 
I realised that existing `complex.parquet` test file possibly doesn't provide 
desired coverage for changes in this PR. Bulk part of changes are aimed at 
making utility functions work with schemas like 
   `a`
   `b`.`a`
   while schema in `complex.parquet` doesn't contain such corner cases.
   Does it make sense to add new resource to cover such cases and base tests on 
that file?


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


With regards,
Apache Git Services