You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by vdiravka <gi...@git.apache.org> on 2017/07/13 18:56:06 UTC

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

GitHub user vdiravka opened a pull request:

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

    DRILL-5660: Drill 1.10 queries fail due to Parquet Metadata "corrupti…

    …on" from DRILL-3867
    
    - bump up metadata file version to v3_1;
    - ignoring of unknown metadata version (for example metadata generated from future versions of drill);
    - adding removing "%20" symbols from path.

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

    $ git pull https://github.com/vdiravka/drill DRILL-5660

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

    https://github.com/apache/drill/pull/877.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 #877
    
----
commit 0d121022e16b5d316c357207706cc1454b77a5a0
Author: Vitalii Diravka <vi...@gmail.com>
Date:   2017-07-12T22:12:00Z

    DRILL-5660: Drill 1.10 queries fail due to Parquet Metadata "corruption" from DRILL-3867
    
    - bump up metadata file version to v3_1;
    - ignoring of unknown metadata version (for example metadata generated from future versions of drill);
    - adding removing "%20" symbols from path.

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128503448
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -723,19 +723,20 @@ private void init(MetadataContext metaContext) throws IOException {
             // if querying a single file we can look up the metadata directly from the file
             metaPath = new Path(p, Metadata.METADATA_FILENAME);
           }
    -      if (metaPath != null && fs.exists(metaPath)) {
    +      if (metaPath != null && fs.exists(metaPath) && Metadata.MetadataVersion.isVersionSupported(fs, metaPath)) {
    --- End diff --
    
    Agree. Thanks. 
    Tthis code is deleted in context of other 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129486097
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    +
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    +    /**
    +     * Version 2: Metadata cache file size is reduced.<br>
    +     * See DRILL-4053
    +     */
    +    public static final String V2 = "v2";
    +    /**
    +     * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +     * Filter pushdown for Parquet is implemented. <br>
    +     * See DRILL-1950
    +     */
    +    public static final String V3 = "v3";
    +    /**
    +     * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +     * See DRILL-3867
    +     */
    +    public static final String V3_1 = "v3.1";
    +
    +
    +    /**
    +     * All historical versions of the Drill metadata cache files
    +     */
    +    public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +    /**
    +     * @param metadataVersion parquet metadata version
    +     * @return true if metadata version is supported, false otherwise
    +     */
    +    public static boolean isVersionSupported(String metadataVersion) {
    +      return SUPPORTED_VERSIONS.contains(metadataVersion);
    +    }
    +
    +    /**
    +     * Helper compare method similar to {@link java.util.Comparator#compare}
    +     *
    +     * @param metadataVersion1 the first metadata version to be compared
    +     * @param metadataVersion2 the second metadata version to be compared
    +     * @return a negative integer, zero, or a positive integer as the
    +     *         first argument is less than, equal to, or greater than the
    +     *         second.
    +     */
    +    public static int compare(String metadataVersion1, String metadataVersion2) {
    --- End diff --
    
    Added new `VersionParser` class, which allow to parse parquet metadata version string. Added test cases for different correct and wrong parquet metadata versions.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r131101557
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.ImmutableSortedSet;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.NavigableSet;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * String metadata version consists of the following characters:<br>
    + * optional "v" letter,<br>
    + * major metadata version (any number of digits),<br>
    + * optional "." delimiter (used if minor metadata version is specified),<br>
    + * minor metadata version (one digit number)
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones. Metadata version value
    +   * doesn't contain `v` letter<br>
    +   * See DRILL-3867, DRILL-5660
    +   */
    +  public static final String V3_1 = "3.1";
    +
    +  /**
    +   * Helper method to parse string metadata version into float.
    +   *
    +   * @param stringVersion text metadata version
    +   * @return parsed Float metadata version
    +   */
    +  public static Float parseStringMetadataVersion(String stringVersion) {
    +    try {
    +      if (stringVersion.contains(".") && stringVersion.split("\\.")[1].length() != 1) {
    +        throw new DrillRuntimeException("Minor metadata version shouldn't be greater than 9 or contain more than one digit");
    +      }
    +      return stringVersion.charAt(0) == 'v' ? Float.valueOf(stringVersion.substring(1)) : Float.valueOf(stringVersion);
    +    } catch (Exception e) {
    +      throw new DrillRuntimeException(String.format("Could not parse metadata version '%s'", stringVersion), e);
    +    }
    +  }
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final NavigableSet<Float> SUPPORTED_VERSIONS = ImmutableSortedSet.of(
    --- End diff --
    
    You can use `SortedSet`, more abstract implementation.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130992403
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -424,32 +432,121 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table dfs_test.tmp.`%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table dfs_test.tmp.`%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata dfs_test.tmp.`%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from dfs_test.tmp.`%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    +      for (MetadataVersion supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    +        if (lastVersion.compareTo(supportedVersion) < 0) {
    +          lastVersion = supportedVersion;
    +        }
    +      }
    +      // Get the future version, which is absent in MetadataVersion.SUPPORTED_VERSIONS list
    +      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.toString().charAt(1))) + 1);
    +      copyMetaDataCacheToTempWithReplacements("parquet/unsupported_metadata/unsupported_metadata_version.requires_replace.txt",
    +          unsupportedMetadataVersion, Metadata.METADATA_FILENAME, futureVersion);
    +      String query = String.format("select * from %s", unsupportedMetadataVersion);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", unsupportedMetadataVersion);
    +    }
    +  }
    +
    +  @Test
    +  public void testCorruptedMetadataFile() throws Exception {
    --- End diff --
    
    Tests were added. 
    And two test cases were added for missing cache files: when root metadata cache file is missing and when inner metadata cache files are absent.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127365605
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    --- End diff --
    
    Here, we can see the reason for the separation. We now open each file twice: once to check the version, another time to deserialize if the version is OK. Better to just deserialize the file. There would be two cases.
    
    * Minor change: the current deserializer can read the file. (This is the case for file version <= code version.) Can also be the case, as here, when the file version bumps without adding new fields.
    * Major change: the deserialization fails with a Jackson exception. This tells us we cannot read the file because we don't recognize the format. This should only be the case when file version > code version.
    
    In either case, we can attempt to deserialize the file:
    
    * Deserialize file.
    * If error occurs, we don't support the file format.
    * If OK, but file version is newer than code version, we don't support.
    * If OK, and file version = code version, this is the normal path.
    * If OK, but file version < code version, then some special fix-up may be needed to convert the deserialized data to current format.
    
    Given this, we don't really need a zillion version classes. We have one class that handles the logic for the current version. We have a deserializer class that handles the above (including any needed data updates.) And, we have the serialized data class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132410703
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -974,8 +1005,17 @@ private int updateRowGroupInfo(long maxRecords) {
       public ParquetGroupScan clone(FileSelection selection) throws IOException {
         ParquetGroupScan newScan = new ParquetGroupScan(this);
         newScan.modifyFileSelection(selection);
    -    newScan.setCacheFileRoot(selection.cacheFileRoot);
    -    newScan.init(selection.getMetaContext());
    +    MetadataContext metaContext = selection.getMetaContext();
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +    }
    +    if (newScan.usedMetadataCache) {
    --- End diff --
    
    Please add / use getter.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129782338
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  @Test
    +  public void testCorrectOnlyMajorVersion() throws Exception {
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(MetadataVersion.Constants.V1);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(1, 0);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectMajorMinorVersions() throws Exception {
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(MetadataVersion.Constants.V3_1);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(3, 1);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testTwoDigitNumberMajorVersion() throws Exception {
    +    String twoDigitNumberMajorVersion = "v10.2";
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(twoDigitNumberMajorVersion);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(10, 2);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectTwoDigitNumberMinorVersion() throws Exception {
    +    String twoDigitNumberMinorVersion = "v3.10";
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(twoDigitNumberMinorVersion);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(3, 10);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test(expected = DrillRuntimeException.class)
    +  public void testVersionWithoutFirstLetter() throws Exception {
    +    String versionWithoutFirstLetter = "3.1";
    +    try {
    +      new MetadataVersion(versionWithoutFirstLetter);
    +    } catch (DrillRuntimeException e) {
    +      assertTrue("Not expected exception is obtained while parsing parquet metadata version",
    +          e.getMessage().contains("Could not parse metadata"));
    +      throw new DrillRuntimeException(e);
    --- End diff --
    
    1. `throw e`, no need to wrap it again in `DrillRuntimeException`
    2. You can check that error message also contains corrupted version:
    `contains(String.format("Could not parse metadata version %s", versionWithoutFirstLetter)));`
    
    Please correct here and in tests below.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128564452
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -446,10 +447,25 @@ public void testMetadataCacheAbsolutePaths() throws Exception {
         }
       }
     
    +  @Test
    --- End diff --
    
    Two test cases are added: `testFutureUnsupportedMetadataVersion()` and `testCorruptedMetadataFile()`. 
    
    Test first one is the test case for the future unsupported versions that is not in the MetadataVersion.SUPPORTED_VERSIONS list. In test case `v4` is used for now (since the last version is `v3_1`). There was `JsonMappingException` earlier. 
    
    Other test case uses metadata file with corrupted json part. There was `JsonParseException` earlier. 
    
    Note: To avoid any `json` deserializing exception we catch `JsonProcessingException` (parent class of the above ones).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130990507
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -495,14 +495,14 @@ public void testFutureUnsupportedMetadataVersion() throws Exception {
         try {
           test("use dfs_test.tmp");
           test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    -      String lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    -      for (String supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    -        if (new MetadataVersion(lastVersion).compareTo(new MetadataVersion(supportedVersion)) < 0) {
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    --- End diff --
    
    Added `ImmutableSortedSet` in the last commit. It allows do not worry about order of elements.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128497912
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1377,7 +1386,7 @@ public void serialize(ColumnMetadata_v2 value, JsonGenerator jgen, SerializerPro
        *
        * Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel
        */
    -  @JsonTypeName("v3") public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase {
    +  @JsonTypeName("v3_1") public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132411564
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -633,13 +645,22 @@ public long getRowCount() {
        * @throws UserException when the updated selection is empty, this happens if the user selects an empty folder.
        */
       private FileSelection
    -  initFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException {
    +  expandSelectionFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException {
         // get the metadata for the root directory by reading the metadata file
         // parquetTableMetadata contains the metadata for all files in the selection root folder, but we need to make sure
         // we only select the files that are part of selection (by setting fileSet appropriately)
     
    +    MetadataContext metaContext = selection.getMetaContext();
    +    // make sure that a metadata context is created since we are going to use metadata caching
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +      selection.setMetaContext(metaContext);
    +    }
         // get (and set internal field) the metadata for the directory by reading the metadata file
    -    this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString(), selection.getMetaContext(), formatConfig);
    +    parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath, metaContext, formatConfig);
    +    if (ignoreExpandingSelection(parquetTableMetadata, selection)) {
    +      return selection;
    +    }
         if (formatConfig.autoCorrectCorruptDates) {
    --- End diff --
    
    Please add / use getter.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127323200
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    --- End diff --
    
    Please add brief method description.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129485798
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java ---
    @@ -639,14 +644,18 @@ protected static void copyDirectoryIntoTempSpace(String resourcesDir, String des
        * @param srcFileOnClassPath the source path of metadata cache file, which should be replaced
        * @param destFolderInTmp  the parent folder name of the metadata cache file
        * @param metaFileName the name of metadata cache file depending on the type of the metadata
    +   * @param customStringReplacement custom string to replace the "CUSTOM_REPLACED" target string in metadata file
        * @throws IOException if a create or write errors occur
        */
    -  protected static void copyMetaDataCacheToTempReplacingInternalPaths(String srcFileOnClassPath, String destFolderInTmp,
    -      String metaFileName) throws IOException {
    +  protected static void copyMetaDataCacheToTempWithReplacements(String srcFileOnClassPath,
    +      String destFolderInTmp, String metaFileName, String customStringReplacement) throws IOException {
         String metadataFileContents = getFile(srcFileOnClassPath);
         Path rootMeta = new Path(dfsTestTmpSchemaLocation, destFolderInTmp);
         Path newMetaCache = new Path(rootMeta, metaFileName);
         FSDataOutputStream outSteam = fs.create(newMetaCache);
    +    if (customStringReplacement!=null) {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128697267
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -38,24 +40,31 @@
       private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
       private static final String tableName1 = "parquetTable1";
       private static final String tableName2 = "parquetTable2";
    -  private static final String RELATIVE_PATHS_METADATA = "relative_paths_metadata";
    +  private static File dataDir1;
    +  private static File dataDir2;
     
     
       @BeforeClass
       public static void copyData() throws Exception {
         // copy the data into the temporary location
         String tmpLocation = getDfsTestTmpSchemaLocation();
    -    File dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    +    dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
         dataDir1.mkdir();
         FileUtils.copyDirectory(new File(String.format(String.format("%s/multilevel/parquet", TEST_RES_PATH))),
             dataDir1);
     
    -    File dataDir2 = new File(tmpLocation + Path.SEPARATOR + tableName2);
    +    dataDir2 = new File(tmpLocation + Path.SEPARATOR + tableName2);
    --- End diff --
    
    The 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128683594
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +41,10 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  // False values of these flags allow to avoid double reading of corrupted or unsupported metadata files
    +  public boolean isMetaCacheFileCorrect = true;
    +  public boolean isMetaDirsCacheFileCorrect = true;
    --- End diff --
    
    Nice!
    
    How are these states cleared? Do we keep track of the file timestamp, and try again if the file is replaced/updated? Or, is there some kind of manual reset? Is the state only per query?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129783958
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    +  public String toString() {
    +    return minor == 0 ? String.format("v%s1", major) : String.format("v%s1.%s2", major, minor);
    +  }
    +
    +  @Override
    +  public int compareTo(MetadataVersion o) {
    +    Preconditions.checkNotNull(o);
    +    return ComparisonChain.start()
    +        .compare(this.major, o.major)
    +        .compare(this.minor, o.minor)
    +        .result();
    +  }
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +  public static class Constants {
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    --- End diff --
    
    As I have pointed out in the previous round of code review, we want to remove the burden from the developer on how parquet metadata version string representation should look like when adding new version. For this `MetadataVersion` class was introduced and `toString` method was overridden:
    `public static final String V1 = new MetadataVersion(1, 0).toString();`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127321261
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    +      ObjectMapper mapper = new ObjectMapper();
    +      mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    +      FSDataInputStream is = fs.open(path);
    --- End diff --
    
    How the steam will be closed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128495021
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -139,13 +139,13 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
        * @return
        * @throws IOException
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129539105
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -424,32 +432,121 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table dfs_test.tmp.`%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table dfs_test.tmp.`%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata dfs_test.tmp.`%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from dfs_test.tmp.`%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      String lastVersion = Iterables.getLast(MetadataVersions.SUPPORTED_VERSIONS);
    +      for (String supportedVersion : MetadataVersions.SUPPORTED_VERSIONS) {
    +        if (MetadataVersions.compare(lastVersion, supportedVersion) < 0) {
    +          lastVersion = supportedVersion;
    +        }
    +      }
    +      // Get the future version, which is absent in MetadataVersions.SUPPORTED_VERSIONS list
    +      String futureVersion = "v" + (Integer.parseInt("" + lastVersion.charAt(1)) + 1);
    --- End diff --
    
    Please user `String.valueOf` instead of concatenating with empty string.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129409026
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring unsupported or corrupted metadata file version. Query performance may be slow. Make sure " +
    +        "the cache file is up-to-date by running the REFRESH TABLE METADATA command");
    +    return null;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, true, metaContext);
    -    return metadata.parquetTableMetadataDirs;
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaDirsCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, true, metaContext);
    +        return metadata.parquetTableMetadataDirs;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaDirsCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring corrupted metadata file. Query performance may be slow. Make sure the cache file" +
    --- End diff --
    
    Static helper method is added to avoid duplicated code. 
    Added path to the file that is corrupt in the error message. And added file path that is ignored in the warning message.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128499798
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128502337
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    --- End diff --
    
    That new deserialization persistence class for reading `metadata` version  is removed. 
    For now we try to deserialize the `metadata` file and in case of getting any inheritor of `JsonProcessingException` ( for example `JsonMappingException` or `JsonParseException`) the `metadata` will be null and will be ignored (with appropriate logging). To avoid of reading such corrupted or unsupported file again that status is stored in `metadata context`. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129796074
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    +  public String toString() {
    +    return minor == 0 ? String.format("v%s1", major) : String.format("v%s1.%s2", major, minor);
    +  }
    +
    +  @Override
    +  public int compareTo(MetadataVersion o) {
    +    Preconditions.checkNotNull(o);
    +    return ComparisonChain.start()
    +        .compare(this.major, o.major)
    +        .compare(this.minor, o.minor)
    +        .result();
    +  }
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +  public static class Constants {
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    --- End diff --
    
    As turned out we these constants will be used as annotation attributes thus they MUST have constant expression.
    In this case we need to make sure that developer has added correct version before Drill start up. Let's keep list of `MetadataVersion` instead of `String` thus we'll be able to validate version constant.
    ```
    public static final List<String> SUPPORTED_VERSIONS = new ArrayList();
    
    static {
      SUPPORTED_VERSIONS.add(new MetadataVersion(V1));
      ...
    
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133198332
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,8 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  private boolean isMetadataCacheCorrupted;
    --- End diff --
    
    Looks better. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132408506
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,25 +140,62 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (ignoreReadingMetadata(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, false, metaContext);
         return metadata.parquetTableMetadata;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    --- End diff --
    
    Please add note explaining the case when we would return null value.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129655324
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    --- End diff --
    
    Constant messages are deleted.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133179047
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,8 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  private boolean isMetadataCacheCorrupted;
    --- End diff --
    
    `metadataCacheCorrupted`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128498212
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    --- End diff --
    
    Answered above. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129103339
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +41,10 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  // False values of these flags allow to avoid double reading of corrupted or unsupported metadata files
    +  public boolean isMetaCacheFileCorrect = true;
    +  public boolean isMetaDirsCacheFileCorrect = true;
    --- End diff --
    
    The state is created after failed opening of the metadata cache file. And will be stored per query.
    There is no case for now to clear that state manually. 
    Note: `MetadataContext.clear()` removes all metadata context info (`isMetadataFilesCorrupted` too).
    
    I left only one state, since if one file is unsupported or corrupted it is more likely that other files are the same. Even if not, it can be dangerous to read some files, but ignore the others.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132534589
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java ---
    @@ -222,15 +222,17 @@ public DrillTable isReadable(DrillFileSystem fs, FileSelection selection,
               // create a metadata context that will be used for the duration of the query for this table
               MetadataContext metaContext = new MetadataContext();
     
    -          ParquetTableMetadataDirs mDirs = Metadata.readMetadataDirs(fs, dirMetaPath.toString(), metaContext, formatConfig);
    -          if (mDirs.getDirectories().size() > 0) {
    -            FileSelection dirSelection = FileSelection.createFromDirectories(mDirs.getDirectories(), selection,
    -                selection.getSelectionRoot() /* cacheFileRoot initially points to selectionRoot */);
    -            dirSelection.setExpandedPartial();
    -            dirSelection.setMetaContext(metaContext);
    -
    -            return new DynamicDrillTable(fsPlugin, storageEngineName, userName,
    -                new FormatSelection(plugin.getConfig(), dirSelection));
    +          ParquetTableMetadataDirs mDirs = Metadata.readMetadataDirs(fs, dirMetaPath, metaContext, formatConfig);
    +          if (mDirs != null) {
    +            if (mDirs.getDirectories().size() > 0) {
    --- End diff --
    
    Missed that. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129834750
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    +  public String toString() {
    +    return minor == 0 ? String.format("v%s1", major) : String.format("v%s1.%s2", major, minor);
    +  }
    +
    +  @Override
    +  public int compareTo(MetadataVersion o) {
    +    Preconditions.checkNotNull(o);
    +    return ComparisonChain.start()
    +        .compare(this.major, o.major)
    +        .compare(this.minor, o.minor)
    +        .result();
    +  }
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +  public static class Constants {
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    --- End diff --
    
    Makes sense. Also started using an `ImmutableList` 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127365148
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    --- End diff --
    
    Not now, but in the future, separate the Jackson-serialized persistence class from the code which works with that data. The persistence class just has the fields for the data and simple related operations. Code that works with the data should be in a separate class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133198419
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -543,78 +584,72 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    -
    -    boolean alreadyCheckedModification = false;
    -    boolean newMetadata = false;
    -
    -    if (metaContext != null) {
    -      alreadyCheckedModification = metaContext.getStatus(parentDirString);
    -    }
    -
    -    if (dirsOnly) {
    -      parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class);
    -      logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    -      timer.stop();
    -      parquetTableMetadataDirs.updateRelativePaths(parentDirString);
    -      if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), p, parentDir, metaContext)) {
    -        parquetTableMetadataDirs =
    -            (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getRight();
    -        newMetadata = true;
    -      }
    -    } else {
    -      parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class);
    -      logger.info("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    -      timer.stop();
    -      if (parquetTableMetadata instanceof ParquetTableMetadata_v3) {
    -        ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(parentDirString);
    -      }
    -      if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), p, parentDir, metaContext)) {
    -        parquetTableMetadata =
    -            (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getLeft();
    -        newMetadata = true;
    -      }
    +    try (FSDataInputStream is = fs.open(path)) {
    +      boolean alreadyCheckedModification = false;
    +      boolean newMetadata = false;
    +        alreadyCheckedModification = metaContext.getStatus(metadataParentDirPath);
    +
    +      if (dirsOnly) {
    +        parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class);
    +        logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    +        timer.stop();
    +        parquetTableMetadataDirs.updateRelativePaths(metadataParentDirPath);
    +        if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), path, metadataParentDir, metaContext)) {
    +          parquetTableMetadataDirs =
    +              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getRight();
    +          newMetadata = true;
    +        }
    +      } else {
    +        parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class);
    +        logger.info("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    +        timer.stop();
    +        if (new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareTo(new MetadataVersion(3, 0)) >= 0) {
    +          ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(metadataParentDirPath);
    +        }
    +        if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), path, metadataParentDir, metaContext)) {
    +          parquetTableMetadata =
    +              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getLeft();
    +          newMetadata = true;
    +        }
     
    -      // DRILL-5009: Remove the RowGroup if it is empty
    -      List<? extends ParquetFileMetadata> files = parquetTableMetadata.getFiles();
    -      for (ParquetFileMetadata file : files) {
    -        List<? extends RowGroupMetadata> rowGroups = file.getRowGroups();
    -        for (Iterator<? extends RowGroupMetadata> iter = rowGroups.iterator(); iter.hasNext(); ) {
    -          RowGroupMetadata r = iter.next();
    -          if (r.getRowCount() == 0) {
    -            iter.remove();
    +        // DRILL-5009: Remove the RowGroup if it is empty
    +        List<? extends ParquetFileMetadata> files = parquetTableMetadata.getFiles();
    +        for (ParquetFileMetadata file : files) {
    +          List<? extends RowGroupMetadata> rowGroups = file.getRowGroups();
    +          for (Iterator<? extends RowGroupMetadata> iter = rowGroups.iterator(); iter.hasNext(); ) {
    +            RowGroupMetadata r = iter.next();
    +            if (r.getRowCount() == 0) {
    +              iter.remove();
    +            }
               }
             }
    -      }
    -
    -    }
     
    -    if (newMetadata && metaContext != null) {
    -      // if new metadata files were created, invalidate the existing metadata context
    -      metaContext.clear();
    +      }
    +      if (newMetadata) {
    +        // if new metadata files were created, invalidate the existing metadata context
    +        metaContext.clear();
    +      }
    +    } catch (IOException e) {
    +      logger.error("Failed to read '{}' metadata file", path, e);
    +      metaContext.setMetadataCacheCorrupted(true);
         }
    -
       }
     
       /**
        * Check if the parquet metadata needs to be updated by comparing the modification time of the directories with
        * the modification time of the metadata file
        *
    -   * @param directories
    -   * @param metaFilePath
    -   * @return
    -   * @throws IOException
    +   * @param directories List of directories
    +   * @param metaFilePath path of parquet metadata cache file
    +   * @return true if metadata needs to be updated, false otherwise
    +   * @throws IOException if some resources are not accessible
        */
    -  private boolean tableModified(List<String> directories, Path metaFilePath,
    -      Path parentDir,
    -      MetadataContext metaContext)
    +  private boolean tableModified(List<String> directories, Path metaFilePath, Path parentDir, MetadataContext metaContext)
           throws IOException {
     
         Stopwatch timer = Stopwatch.createStarted();
     
    -    if (metaContext != null) {
    -      metaContext.setStatus(parentDir.toString());
    -    }
    +    metaContext.setStatus(parentDir.toString());
    --- End diff --
    
    Fixed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132515388
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,9 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  // Setting this value as true allows to avoid double reading of corrupted, unsupported or missing metadata files
    +  public boolean isMetadataFilesMissingOrCorrupted;
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127321352
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    --- End diff --
    
    Please add Java doc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130067083
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -495,14 +495,14 @@ public void testFutureUnsupportedMetadataVersion() throws Exception {
         try {
           test("use dfs_test.tmp");
           test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    -      String lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    -      for (String supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    -        if (new MetadataVersion(lastVersion).compareTo(new MetadataVersion(supportedVersion)) < 0) {
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    --- End diff --
    
    No need to this manually:
    ```
    List<MetadataVersion> supportedVersions = new ArrayList(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    Collections.sort(supportedVersions);
    MetadataVersion lastVersion = supportedVersions.get(supportedVersions.size() - 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133143191
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -452,33 +458,202 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      test("use dfs_test.tmp");
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table `%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata `%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from `%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists `%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      MetadataVersion lastVersion = MetadataVersion.Constants.SUPPORTED_VERSIONS.last();
    +      // Get the future version, which is absent in MetadataVersions.SUPPORTED_VERSIONS set
    +      String futureVersion = new MetadataVersion(lastVersion.getMajor() + 1, 0).toString();
    +      copyMetaDataCacheToTempWithReplacements("parquet/unsupported_metadata/unsupported_metadata_version.requires_replace.txt",
    +          unsupportedMetadataVersion, Metadata.METADATA_FILENAME, futureVersion);
    +      String query = String.format("select * from %s", unsupportedMetadataVersion);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", unsupportedMetadataVersion);
    +    }
    +  }
    +
    +  @Test
    +  public void testCorruptedMetadataFile() throws Exception {
    +    final String corruptedMetadata = "corrupted_metadata";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", corruptedMetadata);
    +      copyMetaDataCacheToTempReplacingInternalPaths("parquet/unsupported_metadata/" +
    +          "corrupted_metadata.requires_replace.txt", corruptedMetadata, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", corruptedMetadata);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", corruptedMetadata);
    +    }
    +  }
    +
    +  @Test
    +  public void testEmptyMetadataFile() throws Exception {
    +    final String emptyMetadataFile = "empty_metadata_file";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", emptyMetadataFile);
    +      copyMetaDataCacheToTempReplacingInternalPaths("parquet/unsupported_metadata/" +
    +          "empty_metadata_file.requires_replace.txt", emptyMetadataFile, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", emptyMetadataFile);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", emptyMetadataFile);
    +    }
    +  }
    +
    +  @Test
    +  public void testRootMetadataFileIsAbsent() throws Exception {
    +    try {
    --- End diff --
    
    Created local tables in these two tests. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132406353
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -452,33 +458,202 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      test("use dfs_test.tmp");
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table `%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata `%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from `%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists `%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      MetadataVersion lastVersion = MetadataVersion.Constants.SUPPORTED_VERSIONS.last();
    +      // Get the future version, which is absent in MetadataVersions.SUPPORTED_VERSIONS set
    +      String futureVersion = new MetadataVersion(lastVersion.getMajor() + 1, 0).toString();
    +      copyMetaDataCacheToTempWithReplacements("parquet/unsupported_metadata/unsupported_metadata_version.requires_replace.txt",
    +          unsupportedMetadataVersion, Metadata.METADATA_FILENAME, futureVersion);
    +      String query = String.format("select * from %s", unsupportedMetadataVersion);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", unsupportedMetadataVersion);
    +    }
    +  }
    +
    +  @Test
    +  public void testCorruptedMetadataFile() throws Exception {
    +    final String corruptedMetadata = "corrupted_metadata";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", corruptedMetadata);
    +      copyMetaDataCacheToTempReplacingInternalPaths("parquet/unsupported_metadata/" +
    +          "corrupted_metadata.requires_replace.txt", corruptedMetadata, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", corruptedMetadata);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", corruptedMetadata);
    +    }
    +  }
    +
    +  @Test
    +  public void testEmptyMetadataFile() throws Exception {
    +    final String emptyMetadataFile = "empty_metadata_file";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", emptyMetadataFile);
    +      copyMetaDataCacheToTempReplacingInternalPaths("parquet/unsupported_metadata/" +
    +          "empty_metadata_file.requires_replace.txt", emptyMetadataFile, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", emptyMetadataFile);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", emptyMetadataFile);
    +    }
    +  }
    +
    +  @Test
    +  public void testRootMetadataFileIsAbsent() throws Exception {
    +    try {
    --- End diff --
    
    It seems that you use the same table for `testRootMetadataFileIsAbsent()` and `testInnerMetadataFilesAreAbsent()` tests, to ensure that tests can be run in parallel they should use different tables.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128697396
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java ---
    @@ -639,14 +644,18 @@ protected static void copyDirectoryIntoTempSpace(String resourcesDir, String des
        * @param srcFileOnClassPath the source path of metadata cache file, which should be replaced
        * @param destFolderInTmp  the parent folder name of the metadata cache file
        * @param metaFileName the name of metadata cache file depending on the type of the metadata
    +   * @param customStringReplacement custom string to replace the "CUSTOM_REPLACED" target string in metadata file
        * @throws IOException if a create or write errors occur
        */
    -  protected static void copyMetaDataCacheToTempReplacingInternalPaths(String srcFileOnClassPath, String destFolderInTmp,
    -      String metaFileName) throws IOException {
    +  protected static void copyMetaDataCacheToTempWithReplacements(String srcFileOnClassPath,
    +      String destFolderInTmp, String metaFileName, String customStringReplacement) throws IOException {
         String metadataFileContents = getFile(srcFileOnClassPath);
         Path rootMeta = new Path(dfsTestTmpSchemaLocation, destFolderInTmp);
         Path newMetaCache = new Path(rootMeta, metaFileName);
         FSDataOutputStream outSteam = fs.create(newMetaCache);
    +    if (customStringReplacement!=null) {
    --- End diff --
    
    `if (customStringReplacement!=null) {` -> `if (customStringReplacement != 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128502875
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    +      ObjectMapper mapper = new ObjectMapper();
    +      mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    +      FSDataInputStream is = fs.open(path);
    +
    +      MetadataVersion metadataVersion = mapper.readValue(is, MetadataVersion.class);
    +      Versions version = Versions.fromString(metadataVersion.textVersion);
    +      if (!(version == null)) {
    --- End diff --
    
    `Enum` and this part of code is removed from 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129528313
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +  /**
    +   * @param metadataVersion parquet metadata version
    +   * @return true if metadata version is supported, false otherwise
    +   */
    +  public static boolean isVersionSupported(String metadataVersion) {
    +    return SUPPORTED_VERSIONS.contains(metadataVersion);
    +  }
    +
    +  /**
    +   * Helper compare method similar to {@link java.util.Comparator#compare}
    +   *
    +   * @param metadataVersion1 the first metadata version to be compared
    +   * @param metadataVersion2 the second metadata version to be compared
    +   * @return a negative integer, zero, or a positive integer as the
    +   *         first argument is less than, equal to, or greater than the
    +   *         second.
    +   */
    +  public static int compare(String metadataVersion1, String metadataVersion2) {
    +    if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +      return VersionParser.parse(metadataVersion1).compareTo(VersionParser.parse(metadataVersion2));
    +    } else {
    +      // this is never reached
    +      throw new DrillRuntimeException(String.format("Unsupported metadata version. '%s' version can't be compared with '%s'",
    +          metadataVersion1, metadataVersion2));
    +    }
    +  }
    +
    +  /**
    +   * Parses a parquet metadata version string
    +   */
    +  public static class VersionParser {
    +    // example: v3.1 or v2
    +    private static final String FORMAT = "v(\\d)\\.?(\\d)?";
    +    private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +    /**
    +     * @param metadataVersion text metadata version
    +     * @return comparable metadata version object
    +     */
    +    public static Version parse(String metadataVersion) {
    +      Matcher matcher = PATTERN.matcher(metadataVersion);
    +      if (!matcher.matches()) {
    +        DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +      }
    +      int majorVersion = Integer.parseInt(matcher.group(1));
    +      int minorVersion = 0;
    +      if (matcher.group(2) != null) {
    +        minorVersion = Integer.parseInt(matcher.group(2));
    +      }
    +      return new Version(metadataVersion, majorVersion, minorVersion, 0, 0, "");
    --- End diff --
    
    Actually you are using `Version` class that is used to present Drill project versions, though its name is generic, it's clearly a different entity that parquet metadata version, it contains such fields like `buildNumber`, `patch` etc. At some point, it can used to compare parquet versions but in reality it's incorrect usage of entity.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127324232
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -557,7 +556,7 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    +    FSDataInputStream is = fs.open(path);
    --- End diff --
    
    How the stream will be closed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133228018
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatConfig.java ---
    @@ -24,9 +24,16 @@
     import com.fasterxml.jackson.annotation.JsonTypeName;
     
     @JsonTypeName("parquet") @JsonInclude(JsonInclude.Include.NON_DEFAULT)
    -public class ParquetFormatConfig implements FormatPluginConfig{
    +public class ParquetFormatConfig implements FormatPluginConfig {
     
    -  public boolean autoCorrectCorruptDates = true;
    +  private boolean autoCorrectCorruptDates = true;
    --- End diff --
    
    4 unit test have failed in `TestCorruptParquetDateCorrection` class.
    For Json generation let's leave `autoCorrectCorruptDates` as public and `areCorruptDatesAutoCorrected()` method mark as `@JsonIgnore`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132413275
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -268,11 +276,15 @@ private FileSelection expandIfNecessary(FileSelection selection) throws IOExcept
         // use the cacheFileRoot if provided (e.g after partition pruning)
         Path metaFilePath = new Path(cacheFileRoot != null ? cacheFileRoot : selectionRoot, Metadata.METADATA_FILENAME);
         if (!fs.exists(metaFilePath)) { // no metadata cache
    +      MetadataContext metaContext = selection.getMetaContext();
    --- End diff --
    
    It seems you set metadata context, when there can be the cases when it is still null? Also can you please explain why there are so many checks in the code if meta context is null, can we make sure it always exists?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129783208
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    --- End diff --
    
    `StringBuilder` will look much cleaner here:
    ``` 
    StringBuilder builder = new StringBuilder();
    builder.append("v").append(major);
    if (minor != 0) {
        builder.append(".").append(minor);
    }
    return builder.toString();
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132405399
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -268,11 +276,15 @@ private FileSelection expandIfNecessary(FileSelection selection) throws IOExcept
         // use the cacheFileRoot if provided (e.g after partition pruning)
         Path metaFilePath = new Path(cacheFileRoot != null ? cacheFileRoot : selectionRoot, Metadata.METADATA_FILENAME);
         if (!fs.exists(metaFilePath)) { // no metadata cache
    +      MetadataContext metaContext = selection.getMetaContext();
    +      if (metaContext != null) {
    +        // some metadata files are absent, but some are present (since metadata context was created)
    +        metaContext.isMetadataFilesMissingOrCorrupted = true;
    --- End diff --
    
    Please use setter & getter instead of direct assignment / access.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132407801
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java ---
    @@ -222,15 +222,17 @@ public DrillTable isReadable(DrillFileSystem fs, FileSelection selection,
               // create a metadata context that will be used for the duration of the query for this table
               MetadataContext metaContext = new MetadataContext();
     
    -          ParquetTableMetadataDirs mDirs = Metadata.readMetadataDirs(fs, dirMetaPath.toString(), metaContext, formatConfig);
    -          if (mDirs.getDirectories().size() > 0) {
    -            FileSelection dirSelection = FileSelection.createFromDirectories(mDirs.getDirectories(), selection,
    -                selection.getSelectionRoot() /* cacheFileRoot initially points to selectionRoot */);
    -            dirSelection.setExpandedPartial();
    -            dirSelection.setMetaContext(metaContext);
    -
    -            return new DynamicDrillTable(fsPlugin, storageEngineName, userName,
    -                new FormatSelection(plugin.getConfig(), dirSelection));
    +          ParquetTableMetadataDirs mDirs = Metadata.readMetadataDirs(fs, dirMetaPath, metaContext, formatConfig);
    +          if (mDirs != null) {
    +            if (mDirs.getDirectories().size() > 0) {
    --- End diff --
    
    Can we combine two `if` statements into one?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128712562
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    +
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    +    /**
    +     * Version 2: Metadata cache file size is reduced.<br>
    +     * See DRILL-4053
    +     */
    +    public static final String V2 = "v2";
    +    /**
    +     * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +     * Filter pushdown for Parquet is implemented. <br>
    +     * See DRILL-1950
    +     */
    +    public static final String V3 = "v3";
    +    /**
    +     * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +     * See DRILL-3867
    +     */
    +    public static final String V3_1 = "v3.1";
    +
    +
    +    /**
    +     * All historical versions of the Drill metadata cache files
    +     */
    +    public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +    /**
    +     * @param metadataVersion parquet metadata version
    +     * @return true if metadata version is supported, false otherwise
    +     */
    +    public static boolean isVersionSupported(String metadataVersion) {
    +      return SUPPORTED_VERSIONS.contains(metadataVersion);
    +    }
    +
    +    /**
    +     * Helper compare method similar to {@link java.util.Comparator#compare}
    +     *
    +     * @param metadataVersion1 the first metadata version to be compared
    +     * @param metadataVersion2 the second metadata version to be compared
    +     * @return a negative integer, zero, or a positive integer as the
    +     *         first argument is less than, equal to, or greater than the
    +     *         second.
    +     */
    +    public static int compare(String metadataVersion1, String metadataVersion2) {
    --- End diff --
    
    Here you compare two versions based on their index in array list. It's fragile approach. First of all developer should always add version to the end of  `SUPPORTED_VERSIONS` lists but how do you ensure that? At least, note should have been added. Though I think it's better to parse version string representation and compare it.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128697735
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring unsupported or corrupted metadata file version. Query performance may be slow. Make sure " +
    +        "the cache file is up-to-date by running the REFRESH TABLE METADATA command");
    +    return null;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, true, metaContext);
    -    return metadata.parquetTableMetadataDirs;
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaDirsCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, true, metaContext);
    +        return metadata.parquetTableMetadataDirs;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    --- End diff --
    
    The 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129485867
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    --- End diff --
    
    Agree. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127363955
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -557,7 +556,7 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    +    FSDataInputStream is = fs.open(path);
    --- End diff --
    
    Common practice is to enclose this in a try-with-resources block, assuming that this stream derives from `AutoCloseable`. Else, we should add our own `try` block and close the file in `finally`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133198372
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,8 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  private boolean isMetadataCacheCorrupted;
    +
       public MetadataContext() {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132432541
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java ---
    @@ -188,6 +188,10 @@ public void setExpandedFully() {
         this.dirStatus = StatusType.EXPANDED_FULLY;
       }
     
    +  public void revertDirStatuses() {
    --- End diff --
    
    The method is not used any more


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132408285
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSelection.java ---
    @@ -188,6 +188,10 @@ public void setExpandedFully() {
         this.dirStatus = StatusType.EXPANDED_FULLY;
       }
     
    +  public void revertDirStatuses() {
    --- End diff --
    
    Can you please add javadoc explaining cases when we would revert directory statuses.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127324650
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    --- End diff --
    
    Can be replaced with `v.getName().equalsIgnoreCase(version)`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130065442
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -495,14 +495,14 @@ public void testFutureUnsupportedMetadataVersion() throws Exception {
         try {
           test("use dfs_test.tmp");
           test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    -      String lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    -      for (String supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    -        if (new MetadataVersion(lastVersion).compareTo(new MetadataVersion(supportedVersion)) < 0) {
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    +      for (MetadataVersion supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    +        if (lastVersion.compareTo(supportedVersion) < 0) {
               lastVersion = supportedVersion;
             }
           }
           // Get the future version, which is absent in MetadataVersion.SUPPORTED_VERSIONS list
    -      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.charAt(1))) + 1);
    +      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.toString().charAt(1))) + 1);
    --- End diff --
    
    `String futureVersion = new MedataVersion(lastVersion.getMajor() + 1, 0).toString();`
    Just add getters for major and minor versions to `MedataVersion` class first.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129530761
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,25 +134,64 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (isMetadataFilesCorrupted(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, false, metaContext);
         return metadata.parquetTableMetadata;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (isMetadataFilesCorrupted(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, true, metaContext);
         return metadata.parquetTableMetadataDirs;
       }
     
    +  /**
    +   * Checking whether metadata is corrupted
    +   *
    +   * @param metaContext metadata context
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @return true if parquet metadata is corrupted, false otherwise
    +   */
    +  private static boolean isMetadataFilesCorrupted(MetadataContext metaContext, Path path) {
    --- End diff --
    
    This method does not only check if metadata files are corrupt, it also warns the user that corrupted metadata files won't be read. Taking into account warn message, it's better to rename method to `ignoreReadingMetadata`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129545316
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,25 +134,64 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (isMetadataFilesCorrupted(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, false, metaContext);
         return metadata.parquetTableMetadata;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (isMetadataFilesCorrupted(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, true, metaContext);
         return metadata.parquetTableMetadataDirs;
       }
     
    +  /**
    +   * Checking whether metadata is corrupted
    +   *
    +   * @param metaContext metadata context
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @return true if parquet metadata is corrupted, false otherwise
    +   */
    +  private static boolean isMetadataFilesCorrupted(MetadataContext metaContext, Path path) {
    --- End diff --
    
    Agree. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133139901
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -974,8 +1005,17 @@ private int updateRowGroupInfo(long maxRecords) {
       public ParquetGroupScan clone(FileSelection selection) throws IOException {
         ParquetGroupScan newScan = new ParquetGroupScan(this);
         newScan.modifyFileSelection(selection);
    -    newScan.setCacheFileRoot(selection.cacheFileRoot);
    -    newScan.init(selection.getMetaContext());
    +    MetadataContext metaContext = selection.getMetaContext();
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +    }
    +    if (newScan.usedMetadataCache) {
    --- End diff --
    
    `usedMetadataCache` isn't used anymore there.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132533745
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.ImmutableSortedSet;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.SortedSet;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +public class MetadataVersion implements Comparable<MetadataVersion> {
    +
    +  private static final String FORMAT = "v?((?!0)\\d+)(\\.(\\d+))?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(3) != null ? Integer.parseInt(matcher.group(3)) : 0;
    +  }
    +
    +  public int getMajor() {
    +    return major;
    +  }
    +
    +  public int getMinor() {
    +    return minor;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) {
    +      return true;
    +    }
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  /**
    +   * @return string representation of the metadata file version, for example: "v1", "v10", "v4.13"
    +   * <p>
    +   * String metadata version consists of the following characters:<p>
    +   * optional "v" letter,<p>
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128500608
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    --- End diff --
    
    Not the issue for now, since new deserialization persistence class is 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130973130
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,147 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.ImmutableList;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +public class MetadataVersion implements Comparable<MetadataVersion> {
    +
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    --- End diff --
    
    Sounds good.  Implemented. 
    Just one restriction - the last minor version is 9, after that increase major version. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133234012
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatConfig.java ---
    @@ -24,9 +24,16 @@
     import com.fasterxml.jackson.annotation.JsonTypeName;
     
     @JsonTypeName("parquet") @JsonInclude(JsonInclude.Include.NON_DEFAULT)
    -public class ParquetFormatConfig implements FormatPluginConfig{
    +public class ParquetFormatConfig implements FormatPluginConfig {
     
    -  public boolean autoCorrectCorruptDates = true;
    +  private boolean autoCorrectCorruptDates = true;
    --- End diff --
    
    Fixed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129525648
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    --- End diff --
    
    Please add final keyword.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129653217
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    +  private static String TEST_ASSERT_EQUALS_MESSAGE = "Parquet metadata version is parsed incorrectly";
    +  private static String TEST_ASSERT_TRUE_MESSAGE = "Not expected exception is obtained while parsing parquet metadata version";
    --- End diff --
    
    constants are removed. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132517116
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,25 +140,62 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (ignoreReadingMetadata(metaContext, path)) {
    +      return null;
    +    }
         Metadata metadata = new Metadata(fs, formatConfig);
         metadata.readBlockMeta(path, false, metaContext);
         return metadata.parquetTableMetadata;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129659864
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -424,32 +432,121 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table dfs_test.tmp.`%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table dfs_test.tmp.`%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata dfs_test.tmp.`%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from dfs_test.tmp.`%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      String lastVersion = Iterables.getLast(MetadataVersions.SUPPORTED_VERSIONS);
    +      for (String supportedVersion : MetadataVersions.SUPPORTED_VERSIONS) {
    +        if (MetadataVersions.compare(lastVersion, supportedVersion) < 0) {
    +          lastVersion = supportedVersion;
    +        }
    +      }
    +      // Get the future version, which is absent in MetadataVersions.SUPPORTED_VERSIONS list
    +      String futureVersion = "v" + (Integer.parseInt("" + lastVersion.charAt(1)) + 1);
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129781397
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    --- End diff --
    
    Please add Java doc describing how string representation is formed. Basically the part of information you have in class description but it's more relevant 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129485531
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -38,24 +40,31 @@
       private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
       private static final String tableName1 = "parquetTable1";
       private static final String tableName2 = "parquetTable2";
    -  private static final String RELATIVE_PATHS_METADATA = "relative_paths_metadata";
    +  private static File dataDir1;
    +  private static File dataDir2;
     
     
       @BeforeClass
       public static void copyData() throws Exception {
         // copy the data into the temporary location
         String tmpLocation = getDfsTestTmpSchemaLocation();
    -    File dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    +    dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127324951
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -139,13 +139,13 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
        * @return
        * @throws IOException
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    +  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    --- End diff --
    
    Please update java doc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129780880
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    --- End diff --
    
    public


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132410745
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -974,8 +1005,17 @@ private int updateRowGroupInfo(long maxRecords) {
       public ParquetGroupScan clone(FileSelection selection) throws IOException {
         ParquetGroupScan newScan = new ParquetGroupScan(this);
         newScan.modifyFileSelection(selection);
    -    newScan.setCacheFileRoot(selection.cacheFileRoot);
    -    newScan.init(selection.getMetaContext());
    +    MetadataContext metaContext = selection.getMetaContext();
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +    }
    +    if (newScan.usedMetadataCache) {
    +      newScan.setCacheFileRoot(selection.cacheFileRoot);
    --- End diff --
    
    Please use getter.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132408041
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,9 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  // Setting this value as true allows to avoid double reading of corrupted, unsupported or missing metadata files
    +  public boolean isMetadataFilesMissingOrCorrupted;
    --- End diff --
    
    Please make private and add setter and getter.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #877: DRILL-5660: Drill 1.10 queries fail due to Parquet Metadat...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/877
  
    +1, LGTM.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128496785
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -658,11 +657,21 @@ private boolean tableModified(List<String> directories, Path metaFilePath,
         return false;
       }
     
    +  /**
    +   * Basic class for parquet metadata. Inheritors of this class are json serializable structures of
    +   * different versions metadata cache files.
    +   *
    --- End diff --
    
    Thanks. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128502487
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    +      ObjectMapper mapper = new ObjectMapper();
    +      mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    +      FSDataInputStream is = fs.open(path);
    --- End diff --
    
    Answered 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128683724
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring unsupported or corrupted metadata file version. Query performance may be slow. Make sure " +
    +        "the cache file is up-to-date by running the REFRESH TABLE METADATA command");
    +    return null;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, true, metaContext);
    -    return metadata.parquetTableMetadataDirs;
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaDirsCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, true, metaContext);
    +        return metadata.parquetTableMetadataDirs;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaDirsCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring corrupted metadata file. Query performance may be slow. Make sure the cache file" +
    --- End diff --
    
    Maybe move the message to constant so it does not have to be maintained in two places? And, maybe add the file or directory that is corrupt?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127365973
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -723,19 +723,20 @@ private void init(MetadataContext metaContext) throws IOException {
             // if querying a single file we can look up the metadata directly from the file
             metaPath = new Path(p, Metadata.METADATA_FILENAME);
           }
    -      if (metaPath != null && fs.exists(metaPath)) {
    +      if (metaPath != null && fs.exists(metaPath) && Metadata.MetadataVersion.isVersionSupported(fs, metaPath)) {
             usedMetadataCache = true;
    -        parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath.toString(), metaContext, formatConfig);
    +        parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath, metaContext, formatConfig);
           } else {
             parquetTableMetadata = Metadata.getParquetTableMetadata(fs, p.toString(), formatConfig);
           }
         } else {
           Path p = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot));
           metaPath = new Path(p, Metadata.METADATA_FILENAME);
    -      if (fs.isDirectory(new Path(selectionRoot)) && fs.exists(metaPath)) {
    +      if (fs.isDirectory(new Path(selectionRoot)) && fs.exists(metaPath)
    --- End diff --
    
    Can we factor out this code so that we do the checks in only one place? Something like a `boolean loadMetadata(...)` method that does all the checks mentioned above?
    
    In general, repeated code = bad. Somebody has to test this code multiple times if it is repeated. Do the unit tests do so?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128697252
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -38,24 +40,31 @@
       private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
       private static final String tableName1 = "parquetTable1";
       private static final String tableName2 = "parquetTable2";
    -  private static final String RELATIVE_PATHS_METADATA = "relative_paths_metadata";
    +  private static File dataDir1;
    +  private static File dataDir2;
     
     
       @BeforeClass
       public static void copyData() throws Exception {
         // copy the data into the temporary location
         String tmpLocation = getDfsTestTmpSchemaLocation();
    -    File dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    +    dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    --- End diff --
    
    Please replace with `new File (tmpLocation, tableName1)`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129562100
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    +  private static String TEST_ASSERT_EQUALS_MESSAGE = "Parquet metadata version is parsed incorrectly";
    +  private static String TEST_ASSERT_TRUE_MESSAGE = "Not expected exception is obtained while parsing parquet metadata version";
    +
    +  @Test
    +  public void testCorrectOnlyMajorVersion() throws Exception {
    +    String correctOnlyMajorVersion = MetadataVersions.V1;
    +    Version parsedMetadataVersion = MetadataVersions.VersionParser.parse(correctOnlyMajorVersion);
    +    Version expectedMetadataVersion = new Version(correctOnlyMajorVersion, 1, 0, 0, 0, "");
    +    assertEquals(TEST_ASSERT_EQUALS_MESSAGE, expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectMajorMinorVersion() throws Exception {
    +    String correctMajorMinorVersion = MetadataVersions.V3_1;
    +    Version parsedMetadataVersion = MetadataVersions.VersionParser.parse(correctMajorMinorVersion);
    +    Version expectedMetadataVersion = new Version(correctMajorMinorVersion, 3, 1, 0, 0, "");
    +    assertEquals(TEST_ASSERT_EQUALS_MESSAGE, expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133143486
  
    --- Diff: exec/java-exec/src/test/resources/parquet/unsupported_metadata/corrupted_metadata.requires_replace.txt ---
    @@ -0,0 +1,41 @@
    +{
    +  "metadata_version" : "v3",
    +  "columnTypeInfo"
    +
    +  CORRUPTED DATA THAT LEADS TO JsonParseException
    +
    +    "path" : "0_0_0.parquet",
    +    "length" : 2424,
    +    "rowGroups" : [ {
    +      "start" : 4,
    +      "length" : 1802,
    +      "rowCount" : 25,
    +      "hostAffinity" : {
    +        "localhost" : 1.0
    +      },
    +      "columns" : [ {
    +        "name" : [ "n_nationkey" ],
    +        "minValue" : 0,
    +        "maxValue" : 24,
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_name" ],
    +        "minValue" : "ALGERIA",
    +        "maxValue" : "VIETNAM",
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_regionkey" ],
    +        "minValue" : 0,
    +        "maxValue" : 4,
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_comment" ],
    +        "minValue" : " haggle. carefully final deposits detect slyly agai",
    +        "maxValue" : "y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be",
    +        "nulls" : 0
    +      } ]
    +    } ]
    +  } ],
    +  "directories" : [ ],
    +  "drillVersion" : "1.11.0-SNAPSHOT"
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127322815
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -658,11 +657,21 @@ private boolean tableModified(List<String> directories, Path metaFilePath,
         return false;
       }
     
    +  /**
    +   * Basic class for parquet metadata. Inheritors of this class are json serializable structures of
    +   * different versions metadata cache files.
    +   *
    +   * Bump up metadata major version if metadata structure is changed.
    +   * Bump up metadata minor version if only metadata content is changed, but metadata structure is the same.
    +   *
    +   * Note: keep metadata versions synchronized with {@link MetadataVersion.Versions}
    +   */
       @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "metadata_version")
       @JsonSubTypes({
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name="v1"),
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v2.class, name="v2"),
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name="v3")
    +      @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name = MetadataVersion.Versions.Constants.V1),
    --- End diff --
    
    Can be replaced with `Version.V1.getName()`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129103929
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128579751
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -557,7 +556,7 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    +    FSDataInputStream is = fs.open(path);
    --- End diff --
    
    Answered under Paul's 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127366251
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -446,10 +447,25 @@ public void testMetadataCacheAbsolutePaths() throws Exception {
         }
       }
     
    +  @Test
    --- End diff --
    
    Tests for the version checks? A bit tricky because you have to create a file and "predict" the next version.
    
    For this, you need a way to get the current version from your `Version` enum. Then do some simple logic to try:
    
    * Add .1 to version number: 3.1 becomes 3.2.
    * Move to next whole version number: 3.1 becomes 4.
    
    Then, create a file with that version and check how this version of the code handles a future version of the file.
    
    To be thorough, also create a file that contains a field that Jackson cannot deserialize. Be sure we properly handle a "hard" version incompatibility as discussed earlier.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128497802
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -658,11 +657,21 @@ private boolean tableModified(List<String> directories, Path metaFilePath,
         return false;
       }
     
    +  /**
    +   * Basic class for parquet metadata. Inheritors of this class are json serializable structures of
    +   * different versions metadata cache files.
    +   *
    +   * Bump up metadata major version if metadata structure is changed.
    +   * Bump up metadata minor version if only metadata content is changed, but metadata structure is the same.
    +   *
    +   * Note: keep metadata versions synchronized with {@link MetadataVersion.Versions}
    +   */
       @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, include = JsonTypeInfo.As.PROPERTY, property = "metadata_version")
       @JsonSubTypes({
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name="v1"),
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v2.class, name="v2"),
    -      @JsonSubTypes.Type(value = ParquetTableMetadata_v3.class, name="v3")
    +      @JsonSubTypes.Type(value = ParquetTableMetadata_v1.class, name = MetadataVersion.Versions.Constants.V1),
    --- End diff --
    
    There is no case to use `getName()` since annotations attribute value must be constant. 
    Looks like using just constants without `enum` is more clear. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127365808
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -723,19 +723,20 @@ private void init(MetadataContext metaContext) throws IOException {
             // if querying a single file we can look up the metadata directly from the file
             metaPath = new Path(p, Metadata.METADATA_FILENAME);
           }
    -      if (metaPath != null && fs.exists(metaPath)) {
    +      if (metaPath != null && fs.exists(metaPath) && Metadata.MetadataVersion.isVersionSupported(fs, metaPath)) {
    --- End diff --
    
    Nit. No need to check `fs.exists` here as we, in effect, check this when trying to open the file. Handle the file not found exception instead.
    
    Note that Drill is a multi-user system. Even with the `exists` check, there is still a race condition where the file can exist during the check, but be deleted by the time we try to open the file. So, since we have to check on open anyway, no need to check 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129834322
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132405168
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,156 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.ImmutableSortedSet;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.SortedSet;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +public class MetadataVersion implements Comparable<MetadataVersion> {
    +
    +  private static final String FORMAT = "v?((?!0)\\d+)(\\.(\\d+))?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(3) != null ? Integer.parseInt(matcher.group(3)) : 0;
    +  }
    +
    +  public int getMajor() {
    +    return major;
    +  }
    +
    +  public int getMinor() {
    +    return minor;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) {
    +      return true;
    +    }
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  /**
    +   * @return string representation of the metadata file version, for example: "v1", "v10", "v4.13"
    +   * <p>
    +   * String metadata version consists of the following characters:<p>
    +   * optional "v" letter,<p>
    --- End diff --
    
    In our case `toString()` method won't return result with "v", so I guess we can remove this sentence.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129653855
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +  /**
    +   * @param metadataVersion parquet metadata version
    +   * @return true if metadata version is supported, false otherwise
    +   */
    +  public static boolean isVersionSupported(String metadataVersion) {
    +    return SUPPORTED_VERSIONS.contains(metadataVersion);
    +  }
    +
    +  /**
    +   * Helper compare method similar to {@link java.util.Comparator#compare}
    +   *
    +   * @param metadataVersion1 the first metadata version to be compared
    +   * @param metadataVersion2 the second metadata version to be compared
    +   * @return a negative integer, zero, or a positive integer as the
    +   *         first argument is less than, equal to, or greater than the
    +   *         second.
    +   */
    +  public static int compare(String metadataVersion1, String metadataVersion2) {
    +    if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +      return VersionParser.parse(metadataVersion1).compareTo(VersionParser.parse(metadataVersion2));
    +    } else {
    +      // this is never reached
    +      throw new DrillRuntimeException(String.format("Unsupported metadata version. '%s' version can't be compared with '%s'",
    +          metadataVersion1, metadataVersion2));
    +    }
    +  }
    +
    +  /**
    +   * Parses a parquet metadata version string
    +   */
    +  public static class VersionParser {
    +    // example: v3.1 or v2
    +    private static final String FORMAT = "v(\\d)\\.?(\\d)?";
    +    private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +    /**
    +     * @param metadataVersion text metadata version
    +     * @return comparable metadata version object
    +     */
    +    public static Version parse(String metadataVersion) {
    +      Matcher matcher = PATTERN.matcher(metadataVersion);
    +      if (!matcher.matches()) {
    +        DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +      }
    +      int majorVersion = Integer.parseInt(matcher.group(1));
    +      int minorVersion = 0;
    +      if (matcher.group(2) != null) {
    +        minorVersion = Integer.parseInt(matcher.group(2));
    +      }
    +      return new Version(metadataVersion, majorVersion, minorVersion, 0, 0, "");
    --- End diff --
    
    `MetadataVersion` is introduced. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132406973
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java ---
    @@ -639,14 +644,18 @@ protected static void copyDirectoryIntoTempSpace(String resourcesDir, String des
        * @param srcFileOnClassPath the source path of metadata cache file, which should be replaced
        * @param destFolderInTmp  the parent folder name of the metadata cache file
        * @param metaFileName the name of metadata cache file depending on the type of the metadata
    +   * @param customStringReplacement custom string to replace the "CUSTOM_REPLACED" target string in metadata file
        * @throws IOException if a create or write errors occur
        */
    -  protected static void copyMetaDataCacheToTempReplacingInternalPaths(String srcFileOnClassPath, String destFolderInTmp,
    -      String metaFileName) throws IOException {
    +  protected static void copyMetaDataCacheToTempWithReplacements(String srcFileOnClassPath,
    +      String destFolderInTmp, String metaFileName, String customStringReplacement) throws IOException {
         String metadataFileContents = getFile(srcFileOnClassPath);
         Path rootMeta = new Path(dfsTestTmpSchemaLocation, destFolderInTmp);
         Path newMetaCache = new Path(rootMeta, metaFileName);
         FSDataOutputStream outSteam = fs.create(newMetaCache);
    +    if (customStringReplacement != null) {
    +      metadataFileContents = metadataFileContents.replace("CUSTOM_STRING_REPLACEMENT", customStringReplacement);
    +    }
         outSteam.writeBytes(metadataFileContents.replace("REPLACED_IN_TEST", dfsTestTmpSchemaLocation));
         outSteam.close();
    --- End diff --
    
    Please use try-with-resources to ensure that stream will be closed in any case.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128698529
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    +
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    +    /**
    +     * Version 2: Metadata cache file size is reduced.<br>
    +     * See DRILL-4053
    +     */
    +    public static final String V2 = "v2";
    +    /**
    +     * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +     * Filter pushdown for Parquet is implemented. <br>
    +     * See DRILL-1950
    +     */
    +    public static final String V3 = "v3";
    +    /**
    +     * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +     * See DRILL-3867
    +     */
    +    public static final String V3_1 = "v3.1";
    +
    +
    +    /**
    +     * All historical versions of the Drill metadata cache files
    +     */
    +    public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +    /**
    +     * @param metadataVersion parquet metadata version
    +     * @return true if metadata version is supported, false otherwise
    +     */
    +    public static boolean isVersionSupported(String metadataVersion) {
    +      return SUPPORTED_VERSIONS.contains(metadataVersion);
    +    }
    +
    +    /**
    +     * Helper compare method similar to {@link java.util.Comparator#compare}
    +     *
    +     * @param metadataVersion1 the first metadata version to be compared
    +     * @param metadataVersion2 the second metadata version to be compared
    +     * @return a negative integer, zero, or a positive integer as the
    +     *         first argument is less than, equal to, or greater than the
    +     *         second.
    +     */
    +    public static int compare(String metadataVersion1, String metadataVersion2) {
    +      if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +        return Integer.compare(SUPPORTED_VERSIONS.indexOf(metadataVersion1), SUPPORTED_VERSIONS.indexOf(metadataVersion2));
    +      } else {
    +        // this is never reached
    +        throw UserException.validationError()
    --- End diff --
    
    Replace please with `DrillRuntimeException`, it's not user's fault if we try to compare with unsupported version.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129834889
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,130 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  @Test
    +  public void testCorrectOnlyMajorVersion() throws Exception {
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(MetadataVersion.Constants.V1);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(1, 0);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectMajorMinorVersions() throws Exception {
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(MetadataVersion.Constants.V3_1);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(3, 1);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testTwoDigitNumberMajorVersion() throws Exception {
    +    String twoDigitNumberMajorVersion = "v10.2";
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(twoDigitNumberMajorVersion);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(10, 2);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectTwoDigitNumberMinorVersion() throws Exception {
    +    String twoDigitNumberMinorVersion = "v3.10";
    +    MetadataVersion parsedMetadataVersion = new MetadataVersion(twoDigitNumberMinorVersion);
    +    MetadataVersion expectedMetadataVersion = new MetadataVersion(3, 10);
    +    assertEquals("Parquet metadata version is parsed incorrectly", expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test(expected = DrillRuntimeException.class)
    +  public void testVersionWithoutFirstLetter() throws Exception {
    +    String versionWithoutFirstLetter = "3.1";
    +    try {
    +      new MetadataVersion(versionWithoutFirstLetter);
    +    } catch (DrillRuntimeException e) {
    +      assertTrue("Not expected exception is obtained while parsing parquet metadata version",
    +          e.getMessage().contains("Could not parse metadata"));
    +      throw new DrillRuntimeException(e);
    --- End diff --
    
    Missed that. Thanks.
    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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128698268
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    --- End diff --
    
    1. Since this class is used in  other classes, I suggest we factor out as separate class. This class has too many inner classes.
    2. It's not an enum, in this case name should be plural - `MetadataVersions`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129834482
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,124 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +class MetadataVersion implements Comparable<MetadataVersion> {
    +  /**
    +   * String version starts from 'v' letter<p>
    +   * First group is major metadata version (any number of digits, except a single zero digit)<p>
    +   * Next character is optional '.' (if minor version is specified)<p>
    +   * Next group is optional, minor metadata version (any number of digits, except a single zero digit)<p>
    +   * Examples of correct metadata versions: v1, v10, v4.13
    +   */
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    +  private static final Pattern PATTERN = Pattern.compile(FORMAT);
    +
    +  private final int major;
    +  private final int minor;
    +
    +  public MetadataVersion(int major, int minor) {
    +    this.major = major;
    +    this.minor = minor;
    +  }
    +
    +  public MetadataVersion(String metadataVersion) {
    +    Matcher matcher = PATTERN.matcher(metadataVersion);
    +    if (!matcher.matches()) {
    +      DrillRuntimeException.format("Could not parse metadata version '%s' using format '%s'", metadataVersion, FORMAT);
    +    }
    +    this.major = Integer.parseInt(matcher.group(1));
    +    this.minor = matcher.group(2) != null ? Integer.parseInt(matcher.group(2)) : 0;
    +  }
    +
    +  @Override
    +  public boolean equals(Object o) {
    +    if (this == o) return true;
    +    if (!(o instanceof MetadataVersion)) {
    +      return false;
    +    }
    +    MetadataVersion that = (MetadataVersion) o;
    +    return this.major == that.major
    +        && this.minor == that.minor;
    +  }
    +
    +  @Override
    +  public int hashCode() {
    +    int result = major;
    +    result = 31 * result + minor;
    +    return result;
    +  }
    +
    +  @Override
    --- End diff --
    
    Looks better. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129485543
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -38,24 +40,31 @@
       private static final String TEST_RES_PATH = WORKING_PATH + "/src/test/resources";
       private static final String tableName1 = "parquetTable1";
       private static final String tableName2 = "parquetTable2";
    -  private static final String RELATIVE_PATHS_METADATA = "relative_paths_metadata";
    +  private static File dataDir1;
    +  private static File dataDir2;
     
     
       @BeforeClass
       public static void copyData() throws Exception {
         // copy the data into the temporary location
         String tmpLocation = getDfsTestTmpSchemaLocation();
    -    File dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
    +    dataDir1 = new File(tmpLocation + Path.SEPARATOR + tableName1);
         dataDir1.mkdir();
         FileUtils.copyDirectory(new File(String.format(String.format("%s/multilevel/parquet", TEST_RES_PATH))),
             dataDir1);
     
    -    File dataDir2 = new File(tmpLocation + Path.SEPARATOR + tableName2);
    +    dataDir2 = new File(tmpLocation + Path.SEPARATOR + tableName2);
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128495285
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -537,13 +537,12 @@ private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p
        * @return
        * @throws IOException
        */
    -  private void readBlockMeta(String path,
    +  private void readBlockMeta(Path path,
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #877: DRILL-5660: Drill 1.10 queries fail due to Parquet Metadat...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on the issue:

    https://github.com/apache/drill/pull/877
  
    @arina-ielchiieva Small fix was made to resolve some regression tests failings. The branch is rebased to the master version.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132409360
  
    --- Diff: exec/java-exec/src/test/resources/parquet/unsupported_metadata/corrupted_metadata.requires_replace.txt ---
    @@ -0,0 +1,41 @@
    +{
    +  "metadata_version" : "v3",
    +  "columnTypeInfo"
    +
    +  CORRUPTED DATA THAT LEADS TO JsonParseException
    +
    +    "path" : "0_0_0.parquet",
    +    "length" : 2424,
    +    "rowGroups" : [ {
    +      "start" : 4,
    +      "length" : 1802,
    +      "rowCount" : 25,
    +      "hostAffinity" : {
    +        "localhost" : 1.0
    +      },
    +      "columns" : [ {
    +        "name" : [ "n_nationkey" ],
    +        "minValue" : 0,
    +        "maxValue" : 24,
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_name" ],
    +        "minValue" : "ALGERIA",
    +        "maxValue" : "VIETNAM",
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_regionkey" ],
    +        "minValue" : 0,
    +        "maxValue" : 4,
    +        "nulls" : 0
    +      }, {
    +        "name" : [ "n_comment" ],
    +        "minValue" : " haggle. carefully final deposits detect slyly agai",
    +        "maxValue" : "y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be",
    +        "nulls" : 0
    +      } ]
    +    } ]
    +  } ],
    +  "directories" : [ ],
    +  "drillVersion" : "1.11.0-SNAPSHOT"
    --- End diff --
    
    May be to Drill version without snapshot here and below?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127364840
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    --- End diff --
    
    One very handy thing to do for each version constant is to list what changed, possibly including the JIRA number for more information:
    
    ```
    /**
     * Version 3.1: Changes the xyz property.
     * File names stored as relative paths.
     * See DRILL-1234.
     */
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133142647
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java ---
    @@ -639,14 +644,18 @@ protected static void copyDirectoryIntoTempSpace(String resourcesDir, String des
        * @param srcFileOnClassPath the source path of metadata cache file, which should be replaced
        * @param destFolderInTmp  the parent folder name of the metadata cache file
        * @param metaFileName the name of metadata cache file depending on the type of the metadata
    +   * @param customStringReplacement custom string to replace the "CUSTOM_REPLACED" target string in metadata file
        * @throws IOException if a create or write errors occur
        */
    -  protected static void copyMetaDataCacheToTempReplacingInternalPaths(String srcFileOnClassPath, String destFolderInTmp,
    -      String metaFileName) throws IOException {
    +  protected static void copyMetaDataCacheToTempWithReplacements(String srcFileOnClassPath,
    +      String destFolderInTmp, String metaFileName, String customStringReplacement) throws IOException {
         String metadataFileContents = getFile(srcFileOnClassPath);
         Path rootMeta = new Path(dfsTestTmpSchemaLocation, destFolderInTmp);
         Path newMetaCache = new Path(rootMeta, metaFileName);
         FSDataOutputStream outSteam = fs.create(newMetaCache);
    +    if (customStringReplacement != null) {
    +      metadataFileContents = metadataFileContents.replace("CUSTOM_STRING_REPLACEMENT", customStringReplacement);
    +    }
         outSteam.writeBytes(metadataFileContents.replace("REPLACED_IN_TEST", dfsTestTmpSchemaLocation));
         outSteam.close();
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130068040
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -424,32 +432,121 @@ public void testMoveCache() throws Exception {
     
       @Test
       public void testMetadataCacheAbsolutePaths() throws Exception {
    +    final String absolutePathsMetadata = "absolute_paths_metadata";
         try {
           test("use dfs_test.tmp");
    -      final String relative_path_metadata_t1 = RELATIVE_PATHS_METADATA + "/t1";
    -      final String relative_path_metadata_t2 = RELATIVE_PATHS_METADATA + "/t2";
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t1);
    -      test("create table `%s` as select * from cp.`tpch/nation.parquet`", relative_path_metadata_t2);
    +      // creating two inner directories to leverage METADATA_DIRECTORIES_FILENAME metadata file as well
    +      final String absolutePathsMetadataT1 = absolutePathsMetadata + "/t1";
    +      final String absolutePathsMetadataT2 = absolutePathsMetadata + "/t2";
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT1);
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", absolutePathsMetadataT2);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_directories_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_DIRECTORIES_FILENAME);
    +          "metadata_directories_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_DIRECTORIES_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths.requires_replace.txt", RELATIVE_PATHS_METADATA, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths.requires_replace.txt", absolutePathsMetadata, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t1.requires_replace.txt", relative_path_metadata_t1, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t1.requires_replace.txt", absolutePathsMetadataT1, Metadata.METADATA_FILENAME);
           copyMetaDataCacheToTempReplacingInternalPaths("parquet/metadata_with_absolute_path/" +
    -          "metadata_table_with_absolute_paths_t2.requires_replace.txt", relative_path_metadata_t2, Metadata.METADATA_FILENAME);
    +          "metadata_table_with_absolute_paths_t2.requires_replace.txt", absolutePathsMetadataT2, Metadata.METADATA_FILENAME);
    +      String query = String.format("select * from %s", absolutePathsMetadata);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), absolutePathsMetadata);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", absolutePathsMetadata);
    +    }
    +  }
     
    -      int rowCount = testSql(String.format("select * from %s", RELATIVE_PATHS_METADATA));
    -      assertEquals("An incorrect result was obtained while querying a table with metadata cache files", 50, rowCount);
    +  @Test
    +  public void testSpacesInMetadataCachePath() throws Exception {
    +    final String pathWithSpaces = "path with spaces";
    +    try {
    +      // creating multilevel table to store path with spaces in both metadata files (METADATA and METADATA_DIRECTORIES)
    +      test("create table dfs_test.tmp.`%s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("create table dfs_test.tmp.`%1$s/%1$s` as select * from cp.`tpch/nation.parquet`", pathWithSpaces);
    +      test("refresh table metadata dfs_test.tmp.`%s`", pathWithSpaces);
    +      checkForMetadataFile(pathWithSpaces);
    +      String query = String.format("select * from dfs_test.tmp.`%s`", pathWithSpaces);
    +      int expectedRowCount = 50;
    +      int expectedNumFiles = 1; // point to selectionRoot since no pruning is done in this query
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=true";
    +      String cacheFileRootPattern = String.format("cacheFileRoot=%s/%s", getDfsTestTmpSchemaLocation(), pathWithSpaces);
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern, cacheFileRootPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists dfs_test.tmp.`%s`", pathWithSpaces);
    +    }
    +  }
    +
    +  @Test
    +  public void testFutureUnsupportedMetadataVersion() throws Exception {
    +    final String unsupportedMetadataVersion = "unsupported_metadata_version";
    +    try {
    +      test("use dfs_test.tmp");
    +      test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    +      for (MetadataVersion supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    +        if (lastVersion.compareTo(supportedVersion) < 0) {
    +          lastVersion = supportedVersion;
    +        }
    +      }
    +      // Get the future version, which is absent in MetadataVersion.SUPPORTED_VERSIONS list
    +      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.toString().charAt(1))) + 1);
    +      copyMetaDataCacheToTempWithReplacements("parquet/unsupported_metadata/unsupported_metadata_version.requires_replace.txt",
    +          unsupportedMetadataVersion, Metadata.METADATA_FILENAME, futureVersion);
    +      String query = String.format("select * from %s", unsupportedMetadataVersion);
    +      int expectedRowCount = 25;
    +      int expectedNumFiles = 1;
    +      int actualRowCount = testSql(query);
    +      assertEquals("An incorrect result was obtained while querying a table with metadata cache files",
    +          expectedRowCount, actualRowCount);
    +      String numFilesPattern = "numFiles=" + expectedNumFiles;
    +      String usedMetaPattern = "usedMetadataFile=false"; // ignoring metadata cache file
    +      PlanTestBase.testPlanMatchingPatterns(query, new String[]{numFilesPattern, usedMetaPattern},
    +          new String[] {"Filter"});
    +    } finally {
    +      test("drop table if exists %s", unsupportedMetadataVersion);
    +    }
    +  }
    +
    +  @Test
    +  public void testCorruptedMetadataFile() throws Exception {
    --- End diff --
    
    Please add tests for the empty metadata file and when at lest one metadata file is missing.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r131164453
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,86 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.ImmutableSortedSet;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.NavigableSet;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * String metadata version consists of the following characters:<br>
    + * optional "v" letter,<br>
    + * major metadata version (any number of digits),<br>
    + * optional "." delimiter (used if minor metadata version is specified),<br>
    + * minor metadata version (one digit number)
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones. Metadata version value
    +   * doesn't contain `v` letter<br>
    +   * See DRILL-3867, DRILL-5660
    +   */
    +  public static final String V3_1 = "3.1";
    +
    +  /**
    +   * Helper method to parse string metadata version into float.
    +   *
    +   * @param stringVersion text metadata version
    +   * @return parsed Float metadata version
    +   */
    +  public static Float parseStringMetadataVersion(String stringVersion) {
    +    try {
    +      if (stringVersion.contains(".") && stringVersion.split("\\.")[1].length() != 1) {
    +        throw new DrillRuntimeException("Minor metadata version shouldn't be greater than 9 or contain more than one digit");
    +      }
    +      return stringVersion.charAt(0) == 'v' ? Float.valueOf(stringVersion.substring(1)) : Float.valueOf(stringVersion);
    +    } catch (Exception e) {
    +      throw new DrillRuntimeException(String.format("Could not parse metadata version '%s'", stringVersion), e);
    +    }
    +  }
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final NavigableSet<Float> SUPPORTED_VERSIONS = ImmutableSortedSet.of(
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128498692
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129535500
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    --- End diff --
    
    private


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128498631
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    --- End diff --
    
    Not the issue for now, this `enum` is 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129526081
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    +  private static String TEST_ASSERT_EQUALS_MESSAGE = "Parquet metadata version is parsed incorrectly";
    +  private static String TEST_ASSERT_TRUE_MESSAGE = "Not expected exception is obtained while parsing parquet metadata version";
    --- End diff --
    
    `TEST_ASSERT_EQUALS_MESSAGE` -> `TEST_ASSERT_EQUALS_ERROR_MESSAGE`
    `TEST_ASSERT_TRUE_MESSAGE` -> `TEST_ASSERT_TRUE_ERROR_MESSAGE`
    Though in tests you may leave error messages hard-coded string for better visibility.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132414323
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -209,9 +211,10 @@ public ParquetGroupScan( //
         this.entries = Lists.newArrayList();
         if (fileSelection.getMetaContext() != null &&
             (fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_STARTED ||
    -          fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_PRUNED)) {
    -      // if pruning was not applicable or was attempted and nothing was pruned, initialize the
    -      // entries with just the selection root instead of the fully expanded list to reduce overhead.
    +          fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_PRUNED ||
    +            fileSelection.getMetaContext().isMetadataFilesMissingOrCorrupted)) {
    --- End diff --
    
    There are 4 checks in `if` statement, may be we can factor out them in separate 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127324972
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -537,13 +537,12 @@ private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p
        * @return
        * @throws IOException
        */
    -  private void readBlockMeta(String path,
    +  private void readBlockMeta(Path path,
    --- End diff --
    
    Please update java doc.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128496657
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -557,7 +556,7 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    +    FSDataInputStream is = fs.open(path);
    --- End diff --
    
    The stream is closed after performing deserializing ObjectMapper.readValue() [link](https://github.com/apache/drill/blob/9cf6faa7aa834c7ba654ce956c8b523ff3464658/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java#L580). But try-with-resources is a good way to be sure the stream is closed in case of getting exception before deserializing.
    Thanks. 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132536020
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -268,11 +276,15 @@ private FileSelection expandIfNecessary(FileSelection selection) throws IOExcept
         // use the cacheFileRoot if provided (e.g after partition pruning)
         Path metaFilePath = new Path(cacheFileRoot != null ? cacheFileRoot : selectionRoot, Metadata.METADATA_FILENAME);
         if (!fs.exists(metaFilePath)) { // no metadata cache
    +      MetadataContext metaContext = selection.getMetaContext();
    +      if (metaContext != null) {
    +        // some metadata files are absent, but some are present (since metadata context was created)
    +        metaContext.isMetadataFilesMissingOrCorrupted = true;
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129525442
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataVersion.java ---
    @@ -0,0 +1,97 @@
    +/*
    + * 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.store.parquet;
    +
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +import org.junit.Test;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertTrue;
    +
    +
    +public class TestParquetMetadataVersion {
    +
    +  private static String TEST_EXCEPTION_MESSAGE = "Failure is expected while parsing the wrong metadata version";
    +  private static String TEST_ASSERT_EQUALS_MESSAGE = "Parquet metadata version is parsed incorrectly";
    +  private static String TEST_ASSERT_TRUE_MESSAGE = "Not expected exception is obtained while parsing parquet metadata version";
    +
    +  @Test
    +  public void testCorrectOnlyMajorVersion() throws Exception {
    +    String correctOnlyMajorVersion = MetadataVersions.V1;
    +    Version parsedMetadataVersion = MetadataVersions.VersionParser.parse(correctOnlyMajorVersion);
    +    Version expectedMetadataVersion = new Version(correctOnlyMajorVersion, 1, 0, 0, 0, "");
    +    assertEquals(TEST_ASSERT_EQUALS_MESSAGE, expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    +  public void testCorrectMajorMinorVersion() throws Exception {
    +    String correctMajorMinorVersion = MetadataVersions.V3_1;
    +    Version parsedMetadataVersion = MetadataVersions.VersionParser.parse(correctMajorMinorVersion);
    +    Version expectedMetadataVersion = new Version(correctMajorMinorVersion, 3, 1, 0, 0, "");
    +    assertEquals(TEST_ASSERT_EQUALS_MESSAGE, expectedMetadataVersion, parsedMetadataVersion);
    +  }
    +
    +  @Test
    --- End diff --
    
    Commonly in ensure that test won't give false-possitive result, exception is re-thrown and checked via junit.
    Example: `@Test(expected = DrillRuntimeException.class)`. Please correct here and in below tests.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132652665
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -209,9 +211,10 @@ public ParquetGroupScan( //
         this.entries = Lists.newArrayList();
         if (fileSelection.getMetaContext() != null &&
             (fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_STARTED ||
    -          fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_PRUNED)) {
    -      // if pruning was not applicable or was attempted and nothing was pruned, initialize the
    -      // entries with just the selection root instead of the fully expanded list to reduce overhead.
    +          fileSelection.getMetaContext().getPruneStatus() == PruneStatus.NOT_PRUNED ||
    +            fileSelection.getMetaContext().isMetadataFilesMissingOrCorrupted)) {
    --- End diff --
    
    Makes sense. I've factored out checks into the new 'checkIfPruningWasApplicable()' 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133159772
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -268,11 +276,15 @@ private FileSelection expandIfNecessary(FileSelection selection) throws IOExcept
         // use the cacheFileRoot if provided (e.g after partition pruning)
         Path metaFilePath = new Path(cacheFileRoot != null ? cacheFileRoot : selectionRoot, Metadata.METADATA_FILENAME);
         if (!fs.exists(metaFilePath)) { // no metadata cache
    +      MetadataContext metaContext = selection.getMetaContext();
    --- End diff --
    
    Started using metadata context as a ParquetGroupScan field.
    The complexity was with initializing entries with selection root. There was a check when metadata context is  null. The fact that it was not null only in case, when metadata directories file is present and using (in that case metadata context was created).
    Added method with javadoc explaining in what cases entries are set as selection root.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133178381
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/MetadataContext.java ---
    @@ -41,6 +42,8 @@
     
       private PruneStatus pruneStatus = PruneStatus.NOT_STARTED;
     
    +  private boolean isMetadataCacheCorrupted;
    +
       public MetadataContext() {
    --- End diff --
    
    1. We can remove this constructor, it exists by default or init `pruneStatus` and `dirModifCheckMap` in it.
    2. Please move enum `PruneStatus` in the end of the class


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132415968
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -1123,4 +1163,22 @@ public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtili
         }
       }
     
    +  /**
    +   * Ignore expanding selection, if metadata is corrupted
    --- End diff --
    
    Please add comment that you not only check should we ignore expanding selection but also setting `parquetTableMetadata` and `fileSet` and `selection.revertDirStatuses()`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #877: DRILL-5660: Drill 1.10 queries fail due to Parquet Metadat...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on the issue:

    https://github.com/apache/drill/pull/877
  
    I wonder if you are very close to solving another long-running problem in Drill: Parquet metadata file corruption (or, at least, the corruption causing queries to fail.)
    
    We have an issue that two Drillbits can write to metadata file concurrently and corrupt it. We also have a case where Drillbit A writes the file while Drillbit B reads it. In both cases, the reading Drillbit can't deserialize the file and "bad things happen."
    
    We need code that can try to deserialize a file, notice that the JSON in the file does not fit our schema, and handle the deserialization exception. If we do that, it is a small step to ignore all forms of read errors: file-not-found, file truncation, etc.
    
    In this case, we give an explicit version error if we can read the file and it has the wrong version. If we can't deserialize the file, we log an error about "file is of a newer version or corrupt."
    
    After this, the corruption can still occur, but the user view is that queries run slow instead of just failing.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133177759
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -543,78 +584,72 @@ private void readBlockMeta(String path,
         mapper.registerModule(serialModule);
         mapper.registerModule(module);
         mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    -    FSDataInputStream is = fs.open(p);
    -
    -    boolean alreadyCheckedModification = false;
    -    boolean newMetadata = false;
    -
    -    if (metaContext != null) {
    -      alreadyCheckedModification = metaContext.getStatus(parentDirString);
    -    }
    -
    -    if (dirsOnly) {
    -      parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class);
    -      logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    -      timer.stop();
    -      parquetTableMetadataDirs.updateRelativePaths(parentDirString);
    -      if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), p, parentDir, metaContext)) {
    -        parquetTableMetadataDirs =
    -            (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getRight();
    -        newMetadata = true;
    -      }
    -    } else {
    -      parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class);
    -      logger.info("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    -      timer.stop();
    -      if (parquetTableMetadata instanceof ParquetTableMetadata_v3) {
    -        ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(parentDirString);
    -      }
    -      if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), p, parentDir, metaContext)) {
    -        parquetTableMetadata =
    -            (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(p.getParent()).toString())).getLeft();
    -        newMetadata = true;
    -      }
    +    try (FSDataInputStream is = fs.open(path)) {
    +      boolean alreadyCheckedModification = false;
    +      boolean newMetadata = false;
    +        alreadyCheckedModification = metaContext.getStatus(metadataParentDirPath);
    +
    +      if (dirsOnly) {
    +        parquetTableMetadataDirs = mapper.readValue(is, ParquetTableMetadataDirs.class);
    +        logger.info("Took {} ms to read directories from directory cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    +        timer.stop();
    +        parquetTableMetadataDirs.updateRelativePaths(metadataParentDirPath);
    +        if (!alreadyCheckedModification && tableModified(parquetTableMetadataDirs.getDirectories(), path, metadataParentDir, metaContext)) {
    +          parquetTableMetadataDirs =
    +              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getRight();
    +          newMetadata = true;
    +        }
    +      } else {
    +        parquetTableMetadata = mapper.readValue(is, ParquetTableMetadataBase.class);
    +        logger.info("Took {} ms to read metadata from cache file", timer.elapsed(TimeUnit.MILLISECONDS));
    +        timer.stop();
    +        if (new MetadataVersion(parquetTableMetadata.getMetadataVersion()).compareTo(new MetadataVersion(3, 0)) >= 0) {
    +          ((ParquetTableMetadata_v3) parquetTableMetadata).updateRelativePaths(metadataParentDirPath);
    +        }
    +        if (!alreadyCheckedModification && tableModified(parquetTableMetadata.getDirectories(), path, metadataParentDir, metaContext)) {
    +          parquetTableMetadata =
    +              (createMetaFilesRecursively(Path.getPathWithoutSchemeAndAuthority(path.getParent()).toString())).getLeft();
    +          newMetadata = true;
    +        }
     
    -      // DRILL-5009: Remove the RowGroup if it is empty
    -      List<? extends ParquetFileMetadata> files = parquetTableMetadata.getFiles();
    -      for (ParquetFileMetadata file : files) {
    -        List<? extends RowGroupMetadata> rowGroups = file.getRowGroups();
    -        for (Iterator<? extends RowGroupMetadata> iter = rowGroups.iterator(); iter.hasNext(); ) {
    -          RowGroupMetadata r = iter.next();
    -          if (r.getRowCount() == 0) {
    -            iter.remove();
    +        // DRILL-5009: Remove the RowGroup if it is empty
    +        List<? extends ParquetFileMetadata> files = parquetTableMetadata.getFiles();
    +        for (ParquetFileMetadata file : files) {
    +          List<? extends RowGroupMetadata> rowGroups = file.getRowGroups();
    +          for (Iterator<? extends RowGroupMetadata> iter = rowGroups.iterator(); iter.hasNext(); ) {
    +            RowGroupMetadata r = iter.next();
    +            if (r.getRowCount() == 0) {
    +              iter.remove();
    +            }
               }
             }
    -      }
    -
    -    }
     
    -    if (newMetadata && metaContext != null) {
    -      // if new metadata files were created, invalidate the existing metadata context
    -      metaContext.clear();
    +      }
    +      if (newMetadata) {
    +        // if new metadata files were created, invalidate the existing metadata context
    +        metaContext.clear();
    +      }
    +    } catch (IOException e) {
    +      logger.error("Failed to read '{}' metadata file", path, e);
    +      metaContext.setMetadataCacheCorrupted(true);
         }
    -
       }
     
       /**
        * Check if the parquet metadata needs to be updated by comparing the modification time of the directories with
        * the modification time of the metadata file
        *
    -   * @param directories
    -   * @param metaFilePath
    -   * @return
    -   * @throws IOException
    +   * @param directories List of directories
    +   * @param metaFilePath path of parquet metadata cache file
    +   * @return true if metadata needs to be updated, false otherwise
    +   * @throws IOException if some resources are not accessible
        */
    -  private boolean tableModified(List<String> directories, Path metaFilePath,
    -      Path parentDir,
    -      MetadataContext metaContext)
    +  private boolean tableModified(List<String> directories, Path metaFilePath, Path parentDir, MetadataContext metaContext)
           throws IOException {
     
         Stopwatch timer = Stopwatch.createStarted();
     
    -    if (metaContext != null) {
    -      metaContext.setStatus(parentDir.toString());
    -    }
    +    metaContext.setStatus(parentDir.toString());
    --- End diff --
    
    `parentDir.toUri().getPath()`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130991076
  
    --- Diff: exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetMetadataCache.java ---
    @@ -495,14 +495,14 @@ public void testFutureUnsupportedMetadataVersion() throws Exception {
         try {
           test("use dfs_test.tmp");
           test("create table `%s` as select * from cp.`tpch/nation.parquet`", unsupportedMetadataVersion);
    -      String lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    -      for (String supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    -        if (new MetadataVersion(lastVersion).compareTo(new MetadataVersion(supportedVersion)) < 0) {
    +      MetadataVersion lastVersion = Iterables.getLast(MetadataVersion.Constants.SUPPORTED_VERSIONS);
    +      for (MetadataVersion supportedVersion : MetadataVersion.Constants.SUPPORTED_VERSIONS) {
    +        if (lastVersion.compareTo(supportedVersion) < 0) {
               lastVersion = supportedVersion;
             }
           }
           // Get the future version, which is absent in MetadataVersion.SUPPORTED_VERSIONS list
    -      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.charAt(1))) + 1);
    +      String futureVersion = "v" + (Integer.parseInt(String.valueOf(lastVersion.toString().charAt(1))) + 1);
    --- End diff --
    
    This code was replaced since float metadata versions are introduced.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127322605
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1377,7 +1386,7 @@ public void serialize(ColumnMetadata_v2 value, JsonGenerator jgen, SerializerPro
        *
        * Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel
        */
    -  @JsonTypeName("v3") public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase {
    +  @JsonTypeName("v3_1") public static class ParquetTableMetadata_v3 extends ParquetTableMetadataBase {
    --- End diff --
    
    Can be replaced with `Version` enum (ex: `Version.V1.getName()`), the same in `ParquetTableMetadataBase` class?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r130179724
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersion.java ---
    @@ -0,0 +1,147 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ComparisonChain;
    +import com.google.common.collect.ImmutableList;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +
    +public class MetadataVersion implements Comparable<MetadataVersion> {
    +
    +  private static final String FORMAT = "v((?!0)\\d+)\\.?((?!0)\\d+)?";
    --- End diff --
    
    To avoid having to parse the "v", might as well just say that the version is a number. Define that we increment the major number (before the decimal point) if the JSON structure changes, we increment the minor number (after the decimal point) of the format is unchanged, but the semantics change (as in the absolute/relative path change.) In general, convert the version to a float and just do normal numeric comparison to determine if the file version is earlier, same as, or later than the code version.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133157793
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -633,13 +645,22 @@ public long getRowCount() {
        * @throws UserException when the updated selection is empty, this happens if the user selects an empty folder.
        */
       private FileSelection
    -  initFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException {
    +  expandSelectionFromMetadataCache(FileSelection selection, Path metaFilePath) throws IOException {
         // get the metadata for the root directory by reading the metadata file
         // parquetTableMetadata contains the metadata for all files in the selection root folder, but we need to make sure
         // we only select the files that are part of selection (by setting fileSet appropriately)
     
    +    MetadataContext metaContext = selection.getMetaContext();
    +    // make sure that a metadata context is created since we are going to use metadata caching
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +      selection.setMetaContext(metaContext);
    +    }
         // get (and set internal field) the metadata for the directory by reading the metadata file
    -    this.parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath.toString(), selection.getMetaContext(), formatConfig);
    +    parquetTableMetadata = Metadata.readBlockMeta(fs, metaFilePath, metaContext, formatConfig);
    +    if (ignoreExpandingSelection(parquetTableMetadata, selection)) {
    +      return selection;
    +    }
         if (formatConfig.autoCorrectCorruptDates) {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128697711
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    --- End diff --
    
    `logger.error("Failed to read metadata", e)`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132533379
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -519,17 +564,16 @@ private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p
       /**
        * Read the parquet metadata from a file
        *
    -   * @param path
    -   * @return
    -   * @throws IOException
    +   * @param path to metadata file
    +   * @param dirsOnly true for {@link Metadata#METADATA_DIRECTORIES_FILENAME}
    +   *                 or false for {@link Metadata#METADATA_FILENAME} files reading
    +   * @param metaContext current metadata context
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  private void readBlockMeta(String path,
    -      boolean dirsOnly,
    -      MetadataContext metaContext) throws IOException {
    +  private void readBlockMeta(Path path, boolean dirsOnly, MetadataContext metaContext) {
         Stopwatch timer = Stopwatch.createStarted();
    -    Path p = new Path(path);
    -    Path parentDir = Path.getPathWithoutSchemeAndAuthority(p.getParent()); // parent directory of the metadata file
    -    String parentDirString = parentDir.toUri().toString(); // string representation for parent directory of the metadata file
    +    Path parentDir = Path.getPathWithoutSchemeAndAuthority(path.getParent()); // parent directory of the metadata file
    +    String parentDirString = parentDir.toUri().getPath(); // string representation for parent directory of the metadata file
    --- End diff --
    
    Possibly it can be confused with above `parentDir`. 
    What about `Path metadataParentDirPath` and `String metadataParentDirStringPath`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #877: DRILL-5660: Drill 1.10 queries fail due to Parquet Metadat...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on the issue:

    https://github.com/apache/drill/pull/877
  
    @vdiravka 
    limiting minor version to 9 is breaking the whole concept of major and minor versions usage. Since when we reach minor max we'll need to bump up the major though structure changes did not occur.
    If using float is not working for major and minor version (I believe you having trouble comparing 1.1 and 1.10) then let's leave our `MetadataVersion` class, just get rid of `v` in version 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129654624
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    --- End diff --
    
    `MetadataVersion` is created.  


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129529192
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    --- End diff --
    
    How developer will know which pattern is used to add versions? If developer adds incorrect version, we will be able to find out if it's correct only during version comparison thus in Drill runtime.  We need to provide a better mechanism to ensure that the next version is correctly formed. Taking into account that in parser you already need some parquet metadata version entity, I guess we can think about creating such entity:
    1. It would have two constructors, one for major and minor versions, the second one for string representation. It would be able to parse string into itself so we can get rid off parser class.
    2. It would be comparable.
    3. It would have `toString` method that will return string representation we actually expect. Example: `new MetadataVersion(3, 1).toString()` -> `v3.1`
    4. `hashCode` and `equals` should be overridden.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129486188
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1922,73 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Supported metadata versions.
    +   * <p>
    +   * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +   */
    +  public static class MetadataVersion {
    +
    +    /**
    +     * Version 1: Introduces parquet file metadata caching.<br>
    +     * See DRILL-2743
    +     */
    +    public static final String V1 = "v1";
    +    /**
    +     * Version 2: Metadata cache file size is reduced.<br>
    +     * See DRILL-4053
    +     */
    +    public static final String V2 = "v2";
    +    /**
    +     * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +     * Filter pushdown for Parquet is implemented. <br>
    +     * See DRILL-1950
    +     */
    +    public static final String V3 = "v3";
    +    /**
    +     * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +     * See DRILL-3867
    +     */
    +    public static final String V3_1 = "v3.1";
    +
    +
    +    /**
    +     * All historical versions of the Drill metadata cache files
    +     */
    +    public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +    /**
    +     * @param metadataVersion parquet metadata version
    +     * @return true if metadata version is supported, false otherwise
    +     */
    +    public static boolean isVersionSupported(String metadataVersion) {
    +      return SUPPORTED_VERSIONS.contains(metadataVersion);
    +    }
    +
    +    /**
    +     * Helper compare method similar to {@link java.util.Comparator#compare}
    +     *
    +     * @param metadataVersion1 the first metadata version to be compared
    +     * @param metadataVersion2 the second metadata version to be compared
    +     * @return a negative integer, zero, or a positive integer as the
    +     *         first argument is less than, equal to, or greater than the
    +     *         second.
    +     */
    +    public static int compare(String metadataVersion1, String metadataVersion2) {
    +      if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +        return Integer.compare(SUPPORTED_VERSIONS.indexOf(metadataVersion1), SUPPORTED_VERSIONS.indexOf(metadataVersion2));
    +      } else {
    +        // this is never reached
    +        throw UserException.validationError()
    --- End diff --
    
    Replaced


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129654257
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +  /**
    +   * @param metadataVersion parquet metadata version
    +   * @return true if metadata version is supported, false otherwise
    +   */
    +  public static boolean isVersionSupported(String metadataVersion) {
    +    return SUPPORTED_VERSIONS.contains(metadataVersion);
    +  }
    +
    +  /**
    +   * Helper compare method similar to {@link java.util.Comparator#compare}
    +   *
    +   * @param metadataVersion1 the first metadata version to be compared
    +   * @param metadataVersion2 the second metadata version to be compared
    +   * @return a negative integer, zero, or a positive integer as the
    +   *         first argument is less than, equal to, or greater than the
    +   *         second.
    +   */
    +  public static int compare(String metadataVersion1, String metadataVersion2) {
    +    if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +      return VersionParser.parse(metadataVersion1).compareTo(VersionParser.parse(metadataVersion2));
    +    } else {
    +      // this is never reached
    +      throw new DrillRuntimeException(String.format("Unsupported metadata version. '%s' version can't be compared with '%s'",
    +          metadataVersion1, metadataVersion2));
    +    }
    +  }
    +
    +  /**
    +   * Parses a parquet metadata version string
    +   */
    +  public static class VersionParser {
    --- End diff --
    
    `VersionParser` is deleted. 
    `MetadataVersion` is introduced with inner class `Constants`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133141382
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -1123,4 +1163,22 @@ public GroupScan applyFilter(LogicalExpression filterExpr, UdfUtilities udfUtili
         }
       }
     
    +  /**
    +   * Ignore expanding selection, if metadata is corrupted
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by paul-rogers <gi...@git.apache.org>.
Github user paul-rogers commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127364108
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -658,11 +657,21 @@ private boolean tableModified(List<String> directories, Path metaFilePath,
         return false;
       }
     
    +  /**
    +   * Basic class for parquet metadata. Inheritors of this class are json serializable structures of
    +   * different versions metadata cache files.
    +   *
    --- End diff --
    
    Sorry, nit. This is Javadoc and is HTML formatted. Please insert <p> to separate paragraphs.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #877: DRILL-5660: Drill 1.10 queries fail due to Parquet Metadat...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on the issue:

    https://github.com/apache/drill/pull/877
  
    @paul-rogers could you please review additional commit. The main changes:
    
    1. All IOExceptions are ignored while deserializing metadata cache files (JsonMappingException, JsonParseException...) with appropriate logging. To avoid of reading such corrupted or unsupported file again that status is stored in metadata context.
    
    2. Creating the metadata context while expanding Selection from Metadata Cache (if it is `null`) allows to always detect the status of partition punning.
    
    3. Two new test cases are added: `testCorruptedMetadataFile()` and `testFutureUnsupportedMetadataVersion()`.
    
    4. Test data created in `@BeforeClass` in `TestParquetMetadataCache` will be removed after executing all tests. 
    
    5. For  `testMetadataCacheAbsolutePaths()` and `testSpacesInMetadataCachePath()` tests  `testPlanMatchingPatterns()` is added.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127320876
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    +        for (Versions v : Versions.values()) {
    +          if (v.version.equalsIgnoreCase(version)) {
    +            return v;
    +          }
    +        }
    +        return null;
    +      }
    +
    +      public static class Constants {
    +        public static final String V1 = "v1";
    +        public static final String V2 = "v2";
    +        public static final String V3 = "v3";
    +        public static final String V3_1 = "v3_1";
    +      }
    +    }
    +
    +    /**
    +     * @param fs current file system
    +     * @param path of metadata cache file
    +     * @return true if metadata version is supported, false otherwise
    +     * @throws IOException if parquet metadata can't be deserialized from the json file
    +     */
    +    public static boolean isVersionSupported(FileSystem fs, Path path) throws IOException {
    +      ObjectMapper mapper = new ObjectMapper();
    +      mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
    +      FSDataInputStream is = fs.open(path);
    +
    +      MetadataVersion metadataVersion = mapper.readValue(is, MetadataVersion.class);
    +      Versions version = Versions.fromString(metadataVersion.textVersion);
    +      if (!(version == null)) {
    --- End diff --
    
    `version != 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129545551
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129104084
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -132,23 +135,57 @@ public static ParquetTableMetadata_v3 getParquetTableMetadata(FileSystem fs,
       }
     
       /**
    -   * Get the parquet metadata for a directory by reading the metadata file
    +   * Get the parquet metadata for the table by reading the metadata file
        *
    -   * @param fs
    +   * @param fs current file system
        * @param path The path to the metadata file, located in the directory that contains the parquet files
    -   * @return
    -   * @throws IOException
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet table metadata
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  public static ParquetTableMetadataBase readBlockMeta(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, false, metaContext);
    -    return metadata.parquetTableMetadata;
    +  public static @Nullable ParquetTableMetadataBase readBlockMeta(FileSystem fs, Path path, MetadataContext metaContext,
    +      ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, false, metaContext);
    +        return metadata.parquetTableMetadata;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    +        metaContext.isMetaCacheFileCorrect = false;
    +      }
    +    }
    +    logger.warn("Ignoring unsupported or corrupted metadata file version. Query performance may be slow. Make sure " +
    +        "the cache file is up-to-date by running the REFRESH TABLE METADATA command");
    +    return null;
       }
     
    -  public static ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, String path, MetadataContext metaContext, ParquetFormatConfig formatConfig) throws IOException {
    -    Metadata metadata = new Metadata(fs, formatConfig);
    -    metadata.readBlockMeta(path, true, metaContext);
    -    return metadata.parquetTableMetadataDirs;
    +  /**
    +   * Get the parquet metadata for all subdirectories by reading the metadata file
    +   *
    +   * @param fs current file system
    +   * @param path The path to the metadata file, located in the directory that contains the parquet files
    +   * @param metaContext metadata context
    +   * @param formatConfig parquet format plugin configs
    +   * @return parquet metadata for a directory
    +   * @throws IOException if metadata file can't be read or updated
    +   */
    +  public static @Nullable ParquetTableMetadataDirs readMetadataDirs(FileSystem fs, Path path,
    +      MetadataContext metaContext, ParquetFormatConfig formatConfig) {
    +    if (metaContext.isMetaDirsCacheFileCorrect) {
    +      Metadata metadata = new Metadata(fs, formatConfig);
    +      try {
    +        metadata.readBlockMeta(path, true, metaContext);
    +        return metadata.parquetTableMetadataDirs;
    +      } catch (IOException e) {
    +        logger.error(e.toString());
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r133140000
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -974,8 +1005,17 @@ private int updateRowGroupInfo(long maxRecords) {
       public ParquetGroupScan clone(FileSelection selection) throws IOException {
         ParquetGroupScan newScan = new ParquetGroupScan(this);
         newScan.modifyFileSelection(selection);
    -    newScan.setCacheFileRoot(selection.cacheFileRoot);
    -    newScan.init(selection.getMetaContext());
    +    MetadataContext metaContext = selection.getMetaContext();
    +    if (metaContext == null) {
    +      metaContext = new MetadataContext();
    +    }
    +    if (newScan.usedMetadataCache) {
    +      newScan.setCacheFileRoot(selection.cacheFileRoot);
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r127322298
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    --- End diff --
    
    It would be good to follow Enum naming convention [1]:
    1. Enum naming is usually singular.
    2. Enum types are usually in upper case.
    3. There is no need to use Constants class. Enum types usually represents a fixed set of constants. You can use `getName()` to get String value.
    
    [1] http://docs.oracle.com/javase/tutorial/java/javaOO/enum.html


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128504130
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java ---
    @@ -723,19 +723,20 @@ private void init(MetadataContext metaContext) throws IOException {
             // if querying a single file we can look up the metadata directly from the file
             metaPath = new Path(p, Metadata.METADATA_FILENAME);
           }
    -      if (metaPath != null && fs.exists(metaPath)) {
    +      if (metaPath != null && fs.exists(metaPath) && Metadata.MetadataVersion.isVersionSupported(fs, metaPath)) {
             usedMetadataCache = true;
    -        parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath.toString(), metaContext, formatConfig);
    +        parquetTableMetadata = Metadata.readBlockMeta(fs, metaPath, metaContext, formatConfig);
           } else {
             parquetTableMetadata = Metadata.getParquetTableMetadata(fs, p.toString(), formatConfig);
           }
         } else {
           Path p = Path.getPathWithoutSchemeAndAuthority(new Path(selectionRoot));
           metaPath = new Path(p, Metadata.METADATA_FILENAME);
    -      if (fs.isDirectory(new Path(selectionRoot)) && fs.exists(metaPath)) {
    +      if (fs.isDirectory(new Path(selectionRoot)) && fs.exists(metaPath)
    --- End diff --
    
    Another approach to detect the unsupported version is used. So it's no longer an issue.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r129526936
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/MetadataVersions.java ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * 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.store.parquet;
    +
    +import com.google.common.collect.Lists;
    +import org.apache.drill.common.Version;
    +import org.apache.drill.common.exceptions.DrillRuntimeException;
    +
    +import java.util.List;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +/**
    + * Supported metadata versions.
    + * <p>
    + * Note: keep them synchronized with {@link Metadata.ParquetTableMetadataBase} versions
    + */
    +public class MetadataVersions {
    +  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Metadata.class);
    +
    +  /**
    +   * Version 1: Introduces parquet file metadata caching.<br>
    +   * See DRILL-2743
    +   */
    +  public static final String V1 = "v1";
    +  /**
    +   * Version 2: Metadata cache file size is reduced.<br>
    +   * See DRILL-4053
    +   */
    +  public static final String V2 = "v2";
    +  /**
    +   * Version 3: Difference between v3 and v2 : min/max, type_length, precision, scale, repetitionLevel, definitionLevel.<br>
    +   * Filter pushdown for Parquet is implemented. <br>
    +   * See DRILL-1950
    +   */
    +  public static final String V3 = "v3";
    +  /**
    +   * Version 3.1: Absolute paths of files and directories are replaced with relative ones.<br>
    +   * See DRILL-3867
    +   */
    +  public static final String V3_1 = "v3.1";
    +
    +  /**
    +   * All historical versions of the Drill metadata cache files
    +   */
    +  public static final List<String> SUPPORTED_VERSIONS = Lists.newArrayList(V1, V2, V3, V3_1);
    +
    +  /**
    +   * @param metadataVersion parquet metadata version
    +   * @return true if metadata version is supported, false otherwise
    +   */
    +  public static boolean isVersionSupported(String metadataVersion) {
    +    return SUPPORTED_VERSIONS.contains(metadataVersion);
    +  }
    +
    +  /**
    +   * Helper compare method similar to {@link java.util.Comparator#compare}
    +   *
    +   * @param metadataVersion1 the first metadata version to be compared
    +   * @param metadataVersion2 the second metadata version to be compared
    +   * @return a negative integer, zero, or a positive integer as the
    +   *         first argument is less than, equal to, or greater than the
    +   *         second.
    +   */
    +  public static int compare(String metadataVersion1, String metadataVersion2) {
    +    if (isVersionSupported(metadataVersion1) && isVersionSupported(metadataVersion2)) {
    +      return VersionParser.parse(metadataVersion1).compareTo(VersionParser.parse(metadataVersion2));
    +    } else {
    +      // this is never reached
    +      throw new DrillRuntimeException(String.format("Unsupported metadata version. '%s' version can't be compared with '%s'",
    +          metadataVersion1, metadataVersion2));
    +    }
    +  }
    +
    +  /**
    +   * Parses a parquet metadata version string
    +   */
    +  public static class VersionParser {
    --- End diff --
    
    This an be extracted into separate class, there is no need to keep it as inner.
    If we create separate entity for parquet metadata version, we might include parse method directly in that class.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by arina-ielchiieva <gi...@git.apache.org>.
Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r132414700
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -519,17 +564,16 @@ private void writeFile(ParquetTableMetadataDirs parquetTableMetadataDirs, Path p
       /**
        * Read the parquet metadata from a file
        *
    -   * @param path
    -   * @return
    -   * @throws IOException
    +   * @param path to metadata file
    +   * @param dirsOnly true for {@link Metadata#METADATA_DIRECTORIES_FILENAME}
    +   *                 or false for {@link Metadata#METADATA_FILENAME} files reading
    +   * @param metaContext current metadata context
    +   * @throws IOException if metadata file can't be read or updated
        */
    -  private void readBlockMeta(String path,
    -      boolean dirsOnly,
    -      MetadataContext metaContext) throws IOException {
    +  private void readBlockMeta(Path path, boolean dirsOnly, MetadataContext metaContext) {
         Stopwatch timer = Stopwatch.createStarted();
    -    Path p = new Path(path);
    -    Path parentDir = Path.getPathWithoutSchemeAndAuthority(p.getParent()); // parent directory of the metadata file
    -    String parentDirString = parentDir.toUri().toString(); // string representation for parent directory of the metadata file
    +    Path parentDir = Path.getPathWithoutSchemeAndAuthority(path.getParent()); // parent directory of the metadata file
    +    String parentDirString = parentDir.toUri().getPath(); // string representation for parent directory of the metadata file
    --- End diff --
    
    `parentDirPath`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #877: DRILL-5660: Drill 1.10 queries fail due to Parquet ...

Posted by vdiravka <gi...@git.apache.org>.
Github user vdiravka commented on a diff in the pull request:

    https://github.com/apache/drill/pull/877#discussion_r128498272
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/Metadata.java ---
    @@ -1851,9 +1860,81 @@ private static String relativize(String baseDir, String childPath) {
               .relativize(fullPathWithoutSchemeAndAuthority.toUri()));
           if (relativeFilePath.isAbsolute()) {
             throw new IllegalStateException(String.format("Path %s is not a subpath of %s.",
    -            basePathWithoutSchemeAndAuthority.toUri().toString(), fullPathWithoutSchemeAndAuthority.toUri().toString()));
    +            basePathWithoutSchemeAndAuthority.toUri().getPath(), fullPathWithoutSchemeAndAuthority.toUri().getPath()));
    +      }
    +      return relativeFilePath.toUri().getPath();
    +    }
    +  }
    +
    +  /**
    +   * Used to identify metadata version by the deserialization "metadata_version" first property
    +   * from the metadata cache file
    +   */
    +  public static class MetadataVersion {
    +    @JsonProperty("metadata_version")
    +    public String textVersion;
    +
    +    /**
    +     * Supported metadata versions.
    +     * Note: keep them synchronized with {@link ParquetTableMetadataBase} versions
    +     */
    +    enum Versions {
    +      v1(Constants.V1),
    +      v2(Constants.V2),
    +      v3(Constants.V3),
    +      v3_1(Constants.V3_1);
    +
    +      private final String version;
    +
    +      Versions(String version) {
    +        this.version = version;
    +      }
    +
    +      public String getVersion() {
    +        return version;
    +      }
    +
    +      public static Versions fromString(String version) {
    --- 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---