You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by "ffacs (via GitHub)" <gi...@apache.org> on 2023/03/23 04:07:37 UTC

[GitHub] [orc] ffacs opened a new pull request, #1449: Fix mismatch behavior with option useTightNumericVector

ffacs opened a new pull request, #1449:
URL: https://github.com/apache/orc/pull/1449

   ### What changes were proposed in this pull request?
   Return LongVectorBatch when create a RowBatch for BOOLEAN type with useTightNumericVector option, instead of ByteVectorBatch.
   
   ### Why are the changes needed?
   If a ByteVectorBatch is returned, BooleanColumnReader/Writer would crash because they only accept LongVectorBatch.
   
   
   ### How was this patch tested?
   no test
   


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on a diff in pull request #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1449:
URL: https://github.com/apache/orc/pull/1449#discussion_r1147027335


##########
c++/src/TypeImpl.cc:
##########
@@ -286,7 +286,9 @@ namespace orc {
                                                               MemoryPool& memoryPool, bool encoded,
                                                               bool useTightNumericVector) const {
     switch (static_cast<int64_t>(kind)) {
-      case BOOLEAN:
+      case BOOLEAN: {

Review Comment:
   I think this PR fixes the bug introduced by `useTightNumericVector` which uses `ByteVectorBatch` for `BOOLEAN` accidentally.
   
   Or should we fix this by supporting `ByteVectorBatch` for `BOOLEAN` type?



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

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

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


[GitHub] [orc] dongjoon-hyun commented on a diff in pull request #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on code in PR #1449:
URL: https://github.com/apache/orc/pull/1449#discussion_r1146779649


##########
c++/src/TypeImpl.cc:
##########
@@ -286,7 +286,9 @@ namespace orc {
                                                               MemoryPool& memoryPool, bool encoded,
                                                               bool useTightNumericVector) const {
     switch (static_cast<int64_t>(kind)) {
-      case BOOLEAN:
+      case BOOLEAN: {

Review Comment:
   According to the PR title, do we need to use `useTightNumericVector` variable?



-- 
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: issues-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 #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #1449:
URL: https://github.com/apache/orc/pull/1449#issuecomment-1485524472

   Thank you, @ffacs and @wgtmac .


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on a diff in pull request #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac commented on code in PR #1449:
URL: https://github.com/apache/orc/pull/1449#discussion_r1148709192


##########
c++/src/ColumnReader.cc:
##########
@@ -148,35 +149,40 @@ namespace orc {
     void seekToRowGroup(std::unordered_map<uint64_t, PositionProvider>& positions) override;
   };
 
-  BooleanColumnReader::BooleanColumnReader(const Type& type, StripeStreams& stripe)
+  template <typename BatchType>
+  BooleanColumnReader<BatchType>::BooleanColumnReader(const Type& type, StripeStreams& stripe)
       : ColumnReader(type, stripe) {
     std::unique_ptr<SeekableInputStream> stream =
         stripe.getStream(columnId, proto::Stream_Kind_DATA, true);
     if (stream == nullptr) throw ParseError("DATA stream not found in Boolean column");
     rle = createBooleanRleDecoder(std::move(stream), metrics);
   }
 
-  BooleanColumnReader::~BooleanColumnReader() {
+  template <typename BatchType>
+  BooleanColumnReader<BatchType>::~BooleanColumnReader() {
     // PASS
   }
 
-  uint64_t BooleanColumnReader::skip(uint64_t numValues) {
+  template <typename BatchType>
+  uint64_t BooleanColumnReader<BatchType>::skip(uint64_t numValues) {
     numValues = ColumnReader::skip(numValues);
     rle->skip(numValues);
     return numValues;
   }
 
-  void BooleanColumnReader::next(ColumnVectorBatch& rowBatch, uint64_t numValues, char* notNull) {
+  template <typename BatchType>
+  void BooleanColumnReader<BatchType>::next(ColumnVectorBatch& rowBatch, uint64_t numValues, char* notNull) {
     ColumnReader::next(rowBatch, numValues, notNull);
-    // Since the byte rle places the output in a char* instead of long*,
-    // we cheat here and use the long* and then expand it in a second pass.
-    int64_t* ptr = dynamic_cast<LongVectorBatch&>(rowBatch).data.data();
+    // Since the byte rle places the output in a char*
+    // we cheat here and use the other type and then expand it in a second pass.

Review Comment:
   ```suggestion
       // Since the byte rle places the output in a char* and BatchType here may be
       // LongVectorBatch with long*. We cheat here in that case and use the long*
       // and then expand it in a second pass.
   ```



##########
c++/src/ColumnWriter.cc:
##########
@@ -667,11 +669,12 @@ namespace orc {
     }
   }
 
-  void BooleanColumnWriter::add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues,
+  template <typename BatchType>
+  void BooleanColumnWriter<BatchType>::add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues,
                                 const char* incomingMask) {
-    LongVectorBatch* byteBatch = dynamic_cast<LongVectorBatch*>(&rowBatch);
+    BatchType* byteBatch = dynamic_cast<BatchType*>(&rowBatch);
     if (byteBatch == nullptr) {
-      throw InvalidArgument("Failed to cast to LongVectorBatch");
+      throw InvalidArgument("Failed to cast to BooleanVectorBatch");

Review Comment:
   The exception message may be incorrect. You may need to `#include <typeinfo>` to use its name field to build the message.



##########
c++/src/ColumnWriter.cc:
##########
@@ -667,11 +669,12 @@ namespace orc {
     }
   }
 
-  void BooleanColumnWriter::add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues,
+  template <typename BatchType>
+  void BooleanColumnWriter<BatchType>::add(ColumnVectorBatch& rowBatch, uint64_t offset, uint64_t numValues,
                                 const char* incomingMask) {
-    LongVectorBatch* byteBatch = dynamic_cast<LongVectorBatch*>(&rowBatch);
+    BatchType* byteBatch = dynamic_cast<BatchType*>(&rowBatch);
     if (byteBatch == nullptr) {
-      throw InvalidArgument("Failed to cast to LongVectorBatch");
+      throw InvalidArgument("Failed to cast to BooleanVectorBatch");

Review Comment:
   https://en.cppreference.com/w/cpp/types/type_info



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac merged pull request #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "wgtmac (via GitHub)" <gi...@apache.org>.
wgtmac merged PR #1449:
URL: https://github.com/apache/orc/pull/1449


-- 
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: issues-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 #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #1449:
URL: https://github.com/apache/orc/pull/1449#issuecomment-1481835244

   cc @wgtmac 


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] ffacs commented on pull request #1449: ORC-1399 [C++] Return LongVectorBatch when createRowBatch for boolean type with useTightNumericVector option

Posted by "ffacs (via GitHub)" <gi...@apache.org>.
ffacs commented on PR #1449:
URL: https://github.com/apache/orc/pull/1449#issuecomment-1482593021

   I`ve uploaded a new patch which supports reading/writing `ByteVectorBatch` for `BOOLEAN` type, please take a look in your free time. @dongjoon-hyun @wgtmac @coderex2522 


-- 
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: issues-unsubscribe@orc.apache.org

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