You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@orc.apache.org by GitBox <gi...@apache.org> on 2021/10/11 07:59:28 UTC

[GitHub] [orc] stiga-huang opened a new pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

stiga-huang opened a new pull request #934:
URL: https://github.com/apache/orc/pull/934


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. File a JIRA issue first and use it as a prefix of your PR title, e.g., `ORC-001: Fix ABC`.
     2. Use your PR title to summarize what this PR proposes instead of describing the problem.
     3. Make PR title and description complete because these will be the permanent commit log.
     4. If possible, provide a concise and reproducible example to reproduce the issue for a faster review.
     5. If the PR is unfinished, use GitHub PR Draft feature.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If there is a discussion in the mailing list, please add the link.
   -->
   This PR fixes inconsistent bloom filter hashing for numeric values between the Java client and C++ client. Before this PR, bloom filters written by Java clients will be incorrectly used in the C++ clients due to the hashing inconsistency. There is an example in the JIRA description.
   After this fix, when reading old ORC files that have bloom filters written by old C++ clients, the bloom filters won't be used.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   In Java codes, hash values are in type "long" which is signed. Bitwise operators like ">>" (shifting) are sign-awared. So we should use "int64_t" instead of "uint64_t" in the corresponding C++ codes.
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Add hashing tests in both Java and C++ codes for numeric values. (TestBloomFilter.java & TestBloomFilter.cc)
   Add hashing tests in C++ codes for string values. (TestMurmur3.cc)
   Add test for reading an ORC file with bloom filters written by C++ client in version 1.6.11. Verified bad bloom filters won't affect the results. (TestReader.cc)
   ORC-1025 will focus on adding more tests.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728665883



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       You are right.
   > This is a private method in RowReaderImpl so it's not public interface.
   
   Ur, can we have a better name?
   > Is it ok to keep this name? 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728669382



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       To be clear, I'm +1 for the following.
   > Or should I rename the member variable to sth else?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728688214



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       Renamed the member to `skipBloomFilters` and  the method to `hasBadBloomFilters()`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728668768



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,34 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    hasBadBloomFilters = checkBadBloomFilters();
+  }
+
+  // Check if the file has inconsistent bloom filters (ORC-1024).
+  bool RowReaderImpl::checkBadBloomFilters() {
+    // Only C++ writer in old releases could have bad bloom filters.
+    if (footer->writer() != ORC_CPP_WRITER) return false;
+    // 'softwareVersion' is added in ORC-984 which is resolved in 1.5.13, 1.6.11, and 1.7.0.

Review comment:
       ```java
   - // 'softwareVersion' is added in ORC-984 which is resolved in 1.5.13, 1.6.11, and 1.7.0.
   + // 'softwareVersion' is added at 1.5.13, 1.6.11, and 1.7.0.
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] wgtmac commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
wgtmac commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727185159



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,18 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    // Check if the file has inconsistent bloom filters.
+    hasBadBloomFilters = false;
+    if (footer->writer() == ORC_CPP_WRITER) {
+      const std::string &fullVersion = footer->softwareversion();

Review comment:
       The softwareversion field was added recently. If the field is unset here, it is probably created by C++ writer on an old version. We should also set hasBadBloomFilters in this case.

##########
File path: c++/src/Reader.cc
##########
@@ -363,7 +381,7 @@ namespace orc {
             throw ParseError("Failed to parse the row index");
           }
           rowIndexes[colId] = rowIndex;
-        } else { // Stream_Kind_BLOOM_FILTER_UTF8
+        } else if (!hasBadBloomFilters) { // Stream_Kind_BLOOM_FILTER_UTF8

Review comment:
       Should java reader do same thing?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r740498834



##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       Unfortunately, this seems to break `CentOS 7` C++ compilation on old C++ versions.
   ```
   [ 75%] Building CXX object c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o
   /root/orc/c++/test/TestMurmur3.cc: In member function 'virtual void orc::TestMurmur3_testHash64_Test::TestBody()':
   /root/orc/c++/test/TestMurmur3.cc:36:53: error: conversion to 'uint32_t {aka unsigned int}' from 'long unsigned int' may alter its value [-Werror=conversion]
        uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;
                                                        ^
   cc1plus: all warnings being treated as errors
   make[2]: *** [c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o] Error 1
   make[1]: *** [c++/test/CMakeFiles/orc-test.dir/all] Error 2
   make: *** [all] Error 2
   FAILED centos7
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727798871



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Shall we remove this because we have `bad_bloom_filter_1.6.11.orc`?

##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       Let's remove this line.

##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,34 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    hasBadBloomFilters = checkBadBloomFilters();
+  }
+
+  // Check if the file has inconsistent bloom filters (ORC-1024).

Review comment:
       Shall we remove `(ORC-1024)` from this sentence?

##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       If this is for the `SNAPSHOT` postfix version check test coverage, `1.6.12-SNAPSHOT` is better.
   
   However, I'd like to recommend to write and read in a test instead of having `-SNAPSHOT` static files.

##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the

Review comment:
       Please remove `(ORC-1024)` from this sentence.

##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       For naming, `hasBadBloomFilters` is better in `Reader.hh`.
   Although I understand that you use `checkXXX` here because you already have a member variable `checkBadBloomFilters`, we had better focus on the public interface more.

##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       For naming, `hasBadBloomFilters` is better in `Reader.hh`.
   Although I understand that you use `checkXXX` here because you already have a member variable `checkBadBloomFilters`, we had better focus on the public interface more importantly.

##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -109,6 +109,11 @@
   // identifies that follow columns bytes must be read
   private boolean needsFollowColumnsRead;
   private final boolean noSelectedVector;
+  // identifies whether the file has bad bloom filters that we should not use (ORC-1024).

Review comment:
       Please remove `(ORC-1024)`.

##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them

Review comment:
       Please remove `(ORC-1024)`.

##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them
+   * in the following reads.
+   * @return true if it has.
+   */
+  private boolean checkBadBloomFilters(OrcProto.Footer footer) {

Review comment:
       Thank you for adding this as `private`.

##########
File path: java/core/src/java/org/apache/orc/util/BloomFilter.java
##########
@@ -246,6 +246,13 @@ public void reset() {
     this.bitSet.clear();
   }
 
+  /**
+   * Helper method that only used for tests. Use default visibility.
+   */
+  boolean testBitSetPos(int pos) {

Review comment:
       If this is used only once in `checkBitSet`, let's move this to a test class like `checkBitSet`.

##########
File path: java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
##########
@@ -411,4 +415,35 @@ public void testGetRawDataSizeFromColIndices() throws Exception {
         ReaderImpl.getRawDataSizeFromColIndices(list, types, stats));
     }
   }
+
+  private void CheckFileWithSargs(String fileName, String softwareVersion)
+      throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path(workDir, fileName);
+    FileSystem fs = path.getFileSystem(conf);
+    try (ReaderImpl reader = (ReaderImpl) OrcFile.createReader(path,
+        OrcFile.readerOptions(conf).filesystem(fs))) {
+      assertEquals(softwareVersion, reader.getSoftwareVersion());
+
+      Reader.Options opt = new Reader.Options();
+      SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(conf);
+      builder.equals("id", PredicateLeaf.Type.LONG, 18000000000L);
+      opt.searchArgument(builder.build(), new String[]{"id"});
+
+      TypeDescription schema = reader.getSchema();
+      VectorizedRowBatch batch = schema.createRowBatch();
+      try (RecordReader rows = reader.rows(opt)) {
+        assertTrue(rows.nextBatch(batch), "No rows read out!");
+        assertEquals(5, batch.size);
+        assertFalse(rows.nextBatch(batch));
+      }
+    }
+  }
+
+  @Test
+  public void testSkipBadBloomFilters() throws IOException {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++ ");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       ditto. Please make a file dynamically if you want to test `-SNAPSHOT` postfix.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728568371



##########
File path: java/core/src/java/org/apache/orc/util/BloomFilter.java
##########
@@ -246,6 +246,13 @@ public void reset() {
     this.bitSet.clear();
   }
 
+  /**
+   * Helper method that only used for tests. Use default visibility.
+   */
+  boolean testBitSetPos(int pos) {

Review comment:
       I'm not sure if I understand the requirement. `bitSet` is a private member of `BloomFilter`. Do you mean changing `bitSet` from private to default visibility so we can use it directly in `TestBloomFilter#checkBitSet`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727806530



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       If this is for the `SNAPSHOT` postfix version check test coverage, `1.6.12-SNAPSHOT` is better.
   
   However, I'd like to recommend to write and read in a test instead of having `-SNAPSHOT` static files.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727809212



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       For naming, `hasBadBloomFilters` is better in `Reader.hh`.
   Although I understand that you use `checkXXX` here because you already have a member variable `checkBadBloomFilters`, we had better focus on the public interface more.

##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       For naming, `hasBadBloomFilters` is better in `Reader.hh`.
   Although I understand that you use `checkXXX` here because you already have a member variable `checkBadBloomFilters`, we had better focus on the public interface more importantly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #934:
URL: https://github.com/apache/orc/pull/934#issuecomment-942052149


   Thank you again, @stiga-huang . I finished my first round review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on pull request #934:
URL: https://github.com/apache/orc/pull/934#issuecomment-940000955


   Sorry for the test failures. I can reproduce it on Ubuntu-20.04 and it only happens in RELEASE builds. I'm still debugging.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on pull request #934:
URL: https://github.com/apache/orc/pull/934#issuecomment-942866979


   > For the following, I believe we need to document the performance regression somewhere. WDTY?
   > 
   > > After this fix, when reading old ORC files that have bloom filters written by old C++ clients, the bloom filters won't be used.
   
   Agree on this. What about writing this in the release notes? Or do we have a docuemnt about known issues?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728563042



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r740513751



##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       https://issues.apache.org/jira/browse/ORC-1043 is filed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r740498834



##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       Unfortunately, this seems to break `CentOS 7` C++ compilation on old C++ versions.
   ```
   [ 75%] Building CXX object c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o
   /root/orc/c++/test/TestMurmur3.cc: In member function 'virtual void orc::TestMurmur3_testHash64_Test::TestBody()':
   /root/orc/c++/test/TestMurmur3.cc:36:53: error: conversion to 'uint32_t {aka unsigned int}' from 'long unsigned int' may alter its value [-Werror=conversion]
        uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;
                                                        ^
   cc1plus: all warnings being treated as errors
   make[2]: *** [c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o] Error 1
   make[1]: *** [c++/test/CMakeFiles/orc-test.dir/all] Error 2
   make: *** [all] Error 2
   FAILED centos7
   ```

##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       https://issues.apache.org/jira/browse/ORC-1043 is filed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727811899



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -109,6 +109,11 @@
   // identifies that follow columns bytes must be read
   private boolean needsFollowColumnsRead;
   private final boolean noSelectedVector;
+  // identifies whether the file has bad bloom filters that we should not use (ORC-1024).

Review comment:
       Please remove `(ORC-1024)`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728662942



##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       @stiga-huang . It seems that you are confused. I didn't say to remove comments. I asked you to remove JIRA ID.
   > I can remove this if you prefer not having this comment.
   
   If you want to be clear, you had better mention ORC version numbers explicitly.
   
   For coding style, I believe we both agree that the best coding style is having clear and clean code and self-describing comments. Adding JIRA IDs means redirecting to something-else due to the lack of clarity. I don't think this code part has that kind complexity.
   
   In addition, if you are adding JIRA IDs at every code blocks, we are going to end up having JIRA IDs for all lines. Moreover, in the worst case, next contributors are going to add another JIRA IDs for the same code blocks on top of the existing JIRA IDs.
   
   WDTY?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728662942



##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       @stiga-huang . It seems that you are confused. I must be clear. Sorry for that. I didn't mean to remove comments. I asked you to remove JIRA ID which is not worth of having.
   > I can remove this if you prefer not having this comment.
   
   If you want to be clear, you had better mention ORC version numbers explicitly.
   
   For coding style, I believe we both agree that the best coding style is having clear and clean code and self-describing comments. Adding JIRA IDs means redirecting to something-else due to the lack of clarity. I don't think this code part has that kind complexity.
   
   In addition, if you are adding JIRA IDs at every code blocks, we are going to end up having JIRA IDs for all lines. Moreover, in the worst case, next contributors are going to add another JIRA IDs for the same code blocks on top of the existing JIRA IDs.
   
   WDTY?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728570859



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the
+     * following reads.
+     * @return true if it has.
+     */
+    bool checkBadBloomFilters();

Review comment:
       Yeah, renaming it to `hasBadBloomFilters()` conflicts with the member variable name. This is a private method in `RowReaderImpl` so it's not public interface. Is it ok to keep this name? Or should I rename the member variable to sth else?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728666731



##########
File path: java/core/src/java/org/apache/orc/util/BloomFilter.java
##########
@@ -246,6 +246,13 @@ public void reset() {
     this.bitSet.clear();
   }
 
+  /**
+   * Helper method that only used for tests. Use default visibility.
+   */
+  boolean testBitSetPos(int pos) {

Review comment:
       Ya, sorry. I was confused here. I missed `this.` part in this method. Never mind~




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727798871



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Shall we remove this because we have `bad_bloom_filter_1.6.11.orc`?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727812713



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them
+   * in the following reads.
+   * @return true if it has.
+   */
+  private boolean checkBadBloomFilters(OrcProto.Footer footer) {

Review comment:
       Thank you for adding this as `private`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727812123



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them

Review comment:
       Please remove `(ORC-1024)`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727883073



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Yeah, this is for test coverage for the `SNAPSHOT` check.
   
   These files are generated by the old codes without the fix. If we generate them using the current codes, bloom filters in them won't have the issue. It seems inevitable to introduce static files...
   
   For `1.6.12-SNAPSHOT` vs `1.6.11-SNAPSHOT`, I think we will backport the fix to 1.6 branch. The next release is 1.6.12. So 1.6.12 won't have the issue. Note that we only skip versions of 1.6.0, 1.6.1, ..., 1.6.11, and 1.7.0. So I think we should use `1.6.11-SNAPSHOT`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727802063



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,34 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    hasBadBloomFilters = checkBadBloomFilters();
+  }
+
+  // Check if the file has inconsistent bloom filters (ORC-1024).

Review comment:
       Shall we remove `(ORC-1024)` from this sentence?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728680205



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       OK, I'll remove line 139.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728686427



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -109,6 +109,11 @@
   // identifies that follow columns bytes must be read
   private boolean needsFollowColumnsRead;
   private final boolean noSelectedVector;
+  // identifies whether the file has bad bloom filters that we should not use (ORC-1024).

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728562783



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the

Review comment:
       Done.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727893588



##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       Sorry that I'm not familiar with the code style here. I thought this can explain why we consider the following versions bad. If one wants to know more, he/she can go to the JIRA(ORC-1024) to understand more details. Otherwise, he/she may need to use `git blame` to find this commit, and then get the JIRA ID.
   
   I think leaving JIRA ids in comments is a good practise in Impala's code style. It helps to explain complex issues with a summarized comment. E.g. https://github.com/apache/impala/blob/45fd3320ad4f68ca86998dff0c9504aa896a278a/be/src/runtime/runtime-state.h#L383-L385
   
   I can remove this if you prefer not having this comment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r740498834



##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       Unfortunately, this seems to break `CentOS 7` C++ compilation on old C++ versions.
   ```
   [ 75%] Building CXX object c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o
   /root/orc/c++/test/TestMurmur3.cc: In member function 'virtual void orc::TestMurmur3_testHash64_Test::TestBody()':
   /root/orc/c++/test/TestMurmur3.cc:36:53: error: conversion to 'uint32_t {aka unsigned int}' from 'long unsigned int' may alter its value [-Werror=conversion]
        uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;
                                                        ^
   cc1plus: all warnings being treated as errors
   make[2]: *** [c++/test/CMakeFiles/orc-test.dir/TestMurmur3.cc.o] Error 1
   make[1]: *** [c++/test/CMakeFiles/orc-test.dir/all] Error 2
   make: *** [all] Error 2
   FAILED centos7
   ```

##########
File path: c++/test/TestMurmur3.cc
##########
@@ -0,0 +1,41 @@
+/**
+ * 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.
+ */
+
+#include "Murmur3.hh"
+#include "wrap/gtest-wrapper.h"
+
+namespace orc {
+
+  // Same test as TestMurmur3#testHashCodeM3_64() in Java codes. Make sure the hash values
+  // are consistent between the Java client and C++ client.
+  // TODO(ORC-1025): Add exhaustive test on more strings.
+  TEST(TestMurmur3, testHash64) {
+    uint8_t origin[] = "It was the best of times, it was the worst of times,"
+                       " it was the age of wisdom, it was the age of foolishness,"
+                       " it was the epoch of belief, it was the epoch of incredulity,"
+                       " it was the season of Light, it was the season of Darkness,"
+                       " it was the spring of hope, it was the winter of despair,"
+                       " we had everything before us, we had nothing before us,"
+                       " we were all going direct to Heaven,"
+                       " we were all going direct the other way.";
+    uint32_t len = sizeof(origin) / sizeof(uint8_t) - 1;

Review comment:
       https://issues.apache.org/jira/browse/ORC-1043 is filed.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728683153



##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       I see. I'm ok to remove the JIRA IDs since this is not that complex.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #934:
URL: https://github.com/apache/orc/pull/934#issuecomment-942052149


   Thank you again, @stiga-huang . I finished my first round review.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727801729



##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       Let's remove this line.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728685005



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,34 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    hasBadBloomFilters = checkBadBloomFilters();
+  }
+
+  // Check if the file has inconsistent bloom filters (ORC-1024).
+  bool RowReaderImpl::checkBadBloomFilters() {
+    // Only C++ writer in old releases could have bad bloom filters.
+    if (footer->writer() != ORC_CPP_WRITER) return false;
+    // 'softwareVersion' is added in ORC-984 which is resolved in 1.5.13, 1.6.11, and 1.7.0.

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on pull request #934:
URL: https://github.com/apache/orc/pull/934#issuecomment-943074029


   I've addressed the comments and updated the JIRA with `Docs Text`.
   Thank for your review, @dongjoon-hyun !


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727665795



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,18 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    // Check if the file has inconsistent bloom filters.
+    hasBadBloomFilters = false;
+    if (footer->writer() == ORC_CPP_WRITER) {
+      const std::string &fullVersion = footer->softwareversion();

Review comment:
       Nice catch! Added the checks.

##########
File path: c++/src/Reader.cc
##########
@@ -363,7 +381,7 @@ namespace orc {
             throw ParseError("Failed to parse the row index");
           }
           rowIndexes[colId] = rowIndex;
-        } else { // Stream_Kind_BLOOM_FILTER_UTF8
+        } else if (!hasBadBloomFilters) { // Stream_Kind_BLOOM_FILTER_UTF8

Review comment:
       Good point! Added checks and tests in Java codes as well.

##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Yeah, this is for test coverage for the `SNAPSHOT` check.
   
   These files are generated by the old codes without the fix. If we generate them using the current codes, bloom filters in them won't have the issue. It seems inevitable to introduce static files...
   
   For `1.6.12-SNAPSHOT` vs `1.6.11-SNAPSHOT`, I think we will backport the fix to 1.6 branch. The next release is 1.6.12. So 1.6.12 won't have the issue. Note that we only skip versions of 1.6.0, 1.6.1, ..., 1.6.11, and 1.7.0. So I think we should use `1.6.11-SNAPSHOT`.

##########
File path: c++/src/Reader.cc
##########
@@ -35,6 +35,12 @@
 #include <set>
 
 namespace orc {
+  // ORC files writen by these versions of cpp writers have inconsistent bloom filter
+  // hashing with the Java codes (ORC-1024). Bloom filters of them should not be used
+  // after we fix ORC-1024.

Review comment:
       Sorry that I'm not familiar with the code style here. I thought this can explain why we consider the following versions bad. If one wants to know more, he/she can go to the JIRA(ORC-1024) to understand more details. Otherwise, he/she may need to use `git blame` to find this commit, and then get the JIRA ID.
   
   I think leaving JIRA ids in comments is a good practise in Impala's code style. It helps to explain complex issues with a summarized comment. E.g. https://github.com/apache/impala/blob/45fd3320ad4f68ca86998dff0c9504aa896a278a/be/src/runtime/runtime-state.h#L383-L385
   
   I can remove this if you prefer not having this comment.

##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them
+   * in the following reads.
+   * @return true if it has.
+   */
+  private boolean checkBadBloomFilters(OrcProto.Footer footer) {

Review comment:
       sure




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727815721



##########
File path: java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
##########
@@ -411,4 +415,35 @@ public void testGetRawDataSizeFromColIndices() throws Exception {
         ReaderImpl.getRawDataSizeFromColIndices(list, types, stats));
     }
   }
+
+  private void CheckFileWithSargs(String fileName, String softwareVersion)
+      throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path(workDir, fileName);
+    FileSystem fs = path.getFileSystem(conf);
+    try (ReaderImpl reader = (ReaderImpl) OrcFile.createReader(path,
+        OrcFile.readerOptions(conf).filesystem(fs))) {
+      assertEquals(softwareVersion, reader.getSoftwareVersion());
+
+      Reader.Options opt = new Reader.Options();
+      SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(conf);
+      builder.equals("id", PredicateLeaf.Type.LONG, 18000000000L);
+      opt.searchArgument(builder.build(), new String[]{"id"});
+
+      TypeDescription schema = reader.getSchema();
+      VectorizedRowBatch batch = schema.createRowBatch();
+      try (RecordReader rows = reader.rows(opt)) {
+        assertTrue(rows.nextBatch(batch), "No rows read out!");
+        assertEquals(5, batch.size);
+        assertFalse(rows.nextBatch(batch));
+      }
+    }
+  }
+
+  @Test
+  public void testSkipBadBloomFilters() throws IOException {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++ ");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       ditto. Please make a file dynamically if you want to test `-SNAPSHOT` postfix.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727815056



##########
File path: java/core/src/java/org/apache/orc/util/BloomFilter.java
##########
@@ -246,6 +246,13 @@ public void reset() {
     this.bitSet.clear();
   }
 
+  /**
+   * Helper method that only used for tests. Use default visibility.
+   */
+  boolean testBitSetPos(int pos) {

Review comment:
       If this is used only once in `checkBitSet`, let's move this to a test class like `checkBitSet`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728562708



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,34 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    hasBadBloomFilters = checkBadBloomFilters();
+  }
+
+  // Check if the file has inconsistent bloom filters (ORC-1024).

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728690192



##########
File path: java/core/src/test/org/apache/orc/impl/TestReaderImpl.java
##########
@@ -411,4 +415,35 @@ public void testGetRawDataSizeFromColIndices() throws Exception {
         ReaderImpl.getRawDataSizeFromColIndices(list, types, stats));
     }
   }
+
+  private void CheckFileWithSargs(String fileName, String softwareVersion)
+      throws IOException {
+    Configuration conf = new Configuration();
+    Path path = new Path(workDir, fileName);
+    FileSystem fs = path.getFileSystem(conf);
+    try (ReaderImpl reader = (ReaderImpl) OrcFile.createReader(path,
+        OrcFile.readerOptions(conf).filesystem(fs))) {
+      assertEquals(softwareVersion, reader.getSoftwareVersion());
+
+      Reader.Options opt = new Reader.Options();
+      SearchArgument.Builder builder = SearchArgumentFactory.newBuilder(conf);
+      builder.equals("id", PredicateLeaf.Type.LONG, 18000000000L);
+      opt.searchArgument(builder.build(), new String[]{"id"});
+
+      TypeDescription schema = reader.getSchema();
+      VectorizedRowBatch batch = schema.createRowBatch();
+      try (RecordReader rows = reader.rows(opt)) {
+        assertTrue(rows.nextBatch(batch), "No rows read out!");
+        assertEquals(5, batch.size);
+        assertFalse(rows.nextBatch(batch));
+      }
+    }
+  }
+
+  @Test
+  public void testSkipBadBloomFilters() throws IOException {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++ ");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Removed the SNAPSHOT test case per the above discussion.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727807566



##########
File path: c++/src/Reader.hh
##########
@@ -179,6 +180,13 @@ namespace orc {
      */
     void seekToRowGroup(uint32_t rowGroupEntryId);
 
+    /**
+     * Check if the file has bad bloom filters (ORC-1024). We will skip using them in the

Review comment:
       Please remove `(ORC-1024)` from this sentence.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] dongjoon-hyun commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r728656986



##########
File path: c++/test/TestReader.cc
##########
@@ -101,4 +101,41 @@ namespace orc {
       900, rowsInCurrentStripe, rowIndexStride, includedRowGroups));
   }
 
+  void CheckFileWithSargs(const char* fileName, const char* softwareVersion) {
+    std::stringstream ss;
+    if(const char* example_dir = std::getenv("ORC_EXAMPLE_DIR")) {
+      ss << example_dir;
+    } else {
+      ss << "../../../examples";
+    }
+    // Read a file with bloom filters written by CPP writer in version 1.6.11.
+    ss << "/" << fileName;
+    ReaderOptions readerOpts;
+    std::unique_ptr<Reader> reader =
+      createReader(readLocalFile(ss.str().c_str()), readerOpts);
+    EXPECT_EQ(WriterId::ORC_CPP_WRITER, reader->getWriterId());
+    EXPECT_EQ(softwareVersion, reader->getSoftwareVersion());
+
+    // Create SearchArgument with a EQUALS predicate which can leverage the bloom filters.
+    RowReaderOptions rowReaderOpts;
+    std::unique_ptr<SearchArgumentBuilder> sarg = SearchArgumentFactory::newBuilder();
+    // Integer value 18000000000 has an inconsistent hash before the fix of ORC-1024.
+    sarg->equals(1, PredicateDataType::LONG,Literal(static_cast<int64_t>(18000000000L)));
+    std::unique_ptr<SearchArgument> final_sarg = sarg->build();
+    rowReaderOpts.searchArgument(std::move(final_sarg));
+    std::unique_ptr<RowReader> rowReader = reader->createRowReader(rowReaderOpts);
+
+    // Make sure bad bloom filters won't affect the results.
+    std::unique_ptr<ColumnVectorBatch> batch =
+      rowReader->createRowBatch(1024);
+    EXPECT_TRUE(rowReader->next(*batch));
+    EXPECT_EQ(5, batch->numElements);
+    EXPECT_FALSE(rowReader->next(*batch));
+  }
+
+  TEST(TestRowReader, testSkipBadBloomFilters) {
+    CheckFileWithSargs("bad_bloom_filter_1.6.11.orc", "ORC C++ 1.6.11");
+    CheckFileWithSargs("bad_bloom_filter_1.6.0.orc", "ORC C++");
+    CheckFileWithSargs("bad_bloom_filter_1.6.11-SNAPSHOT.orc", "ORC C++ 1.6.11-SNAPSHOT");

Review comment:
       Note that this is just a duplication of the test coverage except `-SNAPSHOT` parsing part, @stiga-huang .
   I believe this only causes confusion. If that is the case, please remove it simply. I'm explicitly -1 for adding like this.
   
   > For 1.6.12-SNAPSHOT vs 1.6.11-SNAPSHOT, I think we will backport the fix to 1.6 branch. The next release is 1.6.12. So 1.6.12 won't have the issue.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727665795



##########
File path: c++/src/Reader.cc
##########
@@ -243,6 +249,18 @@ namespace orc {
                                           footer->rowindexstride(),
                                           getWriterVersionImpl(_contents.get())));
     }
+
+    // Check if the file has inconsistent bloom filters.
+    hasBadBloomFilters = false;
+    if (footer->writer() == ORC_CPP_WRITER) {
+      const std::string &fullVersion = footer->softwareversion();

Review comment:
       Nice catch! Added the checks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727665977



##########
File path: c++/src/Reader.cc
##########
@@ -363,7 +381,7 @@ namespace orc {
             throw ParseError("Failed to parse the row index");
           }
           rowIndexes[colId] = rowIndex;
-        } else { // Stream_Kind_BLOOM_FILTER_UTF8
+        } else if (!hasBadBloomFilters) { // Stream_Kind_BLOOM_FILTER_UTF8

Review comment:
       Good point! Added checks and tests in Java codes as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [orc] stiga-huang commented on a change in pull request #934: ORC-1024: [C++] Fix inconsistent bloom filter hashing for numeric values

Posted by GitBox <gi...@apache.org>.
stiga-huang commented on a change in pull request #934:
URL: https://github.com/apache/orc/pull/934#discussion_r727896504



##########
File path: java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
##########
@@ -350,6 +356,33 @@ protected RecordReaderImpl(ReaderImpl fileReader,
     }
   }
 
+  /**
+   * Check if the file has inconsistent bloom filters (ORC-1024). We will skip using them
+   * in the following reads.
+   * @return true if it has.
+   */
+  private boolean checkBadBloomFilters(OrcProto.Footer footer) {

Review comment:
       sure




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@orc.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org