You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2020/11/12 03:41:00 UTC

[GitHub] [arrow] mathyingzhou opened a new pull request #8648: ARROW-7906: [C++] Add ORC write support

mathyingzhou opened a new pull request #8648:
URL: https://github.com/apache/arrow/pull/8648


   This pull request tracks the progress on adding ORC write support. The functionality is not complete yet. However for most types the process of populating a ColumnVectorBatch in ORC using data from Arrow Array.
   
   Arrow data types (arrow::Type::type) I do support:
   Boolean: BOOL
   Numerical: INT8, INT16, INT32, INT64, FLOAT, DOUBLE
   Time-related: DATE32
   Binary: BINARY, STRING, LARGE_BINARY, LARGE_STRING, FIXED_SIZE_BINARY
   Nested: LIST, LARGE_LIST, FIXED_SIZE_LIST, STRUCT, MAP, DENSE_UNION, SPARSE_UNION
   
   Arrow data types I plan to support:
   Numerical: DECIMAL128
   Time-related: DATE64, TIMESTAMP
   Dictionary: DICTIONARY
   
   Arrow data types I currently do NOT plan to support:
   Numerical: UINT8, UINT16, UINT32, UINT64, HALF_FLOAT, DECIMAL256 (There are no corresponding types in ORC. Of course except for in the case of DECIMAL256 we can always cast them into larger types. However I think maybe users need to explicitly do that.)
   Time-related: TIME32, TIME64, INTERVAL_MONTHS, INTERVAL_DAY_TIME, DURATION (There are no corresponding types in ORC and it is impossible to cast them into ORC types without losing time-related information)
   Extension: EXTENSION 


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567583923



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {

Review comment:
       Would it be possible to use the util::Bitmap class?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790615411


   @pitrou It is due to some new issue in Decimal128s.  Starting from https://github.com/apache/arrow/pull/8648/commits/68fd76fc5eba3350f341eb0fd7e4f83f7c83c51e we suddenly began to get "NotImplemented: random decimal128 generation with precision > 18". I reduced precision to 18 which caused ORC to misbehave. Shall random decimal128 generation with precision > 18 be impossible? It used to be perfectly fine.
   


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-798971411


   @pitrou Could you please check again? My next PR is going to be ready soon and it is dependent on this one merging to be clean.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990512



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BOOLEAN);
+}
+TEST(TestAdapterWriteConverter, typeInt8) {
+  DataType* type = int8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BYTE);
+}
+TEST(TestAdapterWriteConverter, typeInt16) {
+  DataType* type = int16().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::SHORT);
+}
+TEST(TestAdapterWriteConverter, typeInt32) {
+  DataType* type = int32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeInt64) {
+  DataType* type = int64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LONG);
+}
+TEST(TestAdapterWriteConverter, typeFloat) {
+  DataType* type = float32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::FLOAT);
+}
+TEST(TestAdapterWriteConverter, typeDouble) {
+  DataType* type = float64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DOUBLE);
+}
+TEST(TestAdapterWriteConverter, typeString) {
+  DataType* type = utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeLargeString) {
+  DataType* type = large_utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeBinary) {
+  DataType* type = binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeLargeBinary) {
+  DataType* type = large_binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinary) {
+  DataType* type = fixed_size_binary(3).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinaryZero) {
+  DataType* type = fixed_size_binary(0).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeDate32) {
+  DataType* type = date32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, typeDate64) {
+  DataType* type = date64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampSecond) {
+  DataType* type = timestamp(TimeUnit::type::SECOND).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMilli) {
+  DataType* type = timestamp(TimeUnit::type::MILLI).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMicro) {
+  DataType* type = timestamp(TimeUnit::type::MICRO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampNano) {
+  DataType* type = timestamp(TimeUnit::type::NANO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeDecimal) {
+  DataType* type = decimal(32, 5).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getPrecision(), 32);
+  EXPECT_EQ(out->getScale(), 5);
+}
+TEST(TestAdapterWriteConverter, typeList) {
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeLargeList) {
+  auto sharedPtrArrowType = large_list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeList) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 3);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeListZero) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 0);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeStructTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, typeStructSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeStruct) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getFieldName(1), "b");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeMap) {
+  auto sharedPtrArrowType = map(utf8(), int32());
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::MAP);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnionTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnionSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnion) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnionTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnionSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnion) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfList) {
+  auto sharedPtrArrowSubtype = list(std::make_shared<Field>("a", int32()));
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfMap) {
+  auto sharedPtrArrowSubtype = map(utf8(), int32());
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::MAP);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfStructOfLists) {
+  auto sharedPtrArrowSubsubtype0 = list(std::make_shared<Field>("a", int8()));
+  auto sharedPtrArrowSubsubtype1 = list(std::make_shared<Field>("b", float64()));
+  auto sharedPtrArrowSubsubtype2 = list(std::make_shared<Field>("c", date32()));
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", sharedPtrArrowSubsubtype0));
+  xFields.push_back(std::make_shared<Field>("b", sharedPtrArrowSubsubtype1));
+  xFields.push_back(std::make_shared<Field>("c", sharedPtrArrowSubsubtype2));
+  auto sharedPtrArrowSubtype = struct_(xFields);
+  auto sharedPtrArrowType = list(std::make_shared<Field>("x", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 3);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtype(0)->getKind(),
+            liborc::TypeKind::BYTE);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getSubtype(0)->getKind(),
+            liborc::TypeKind::DOUBLE);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getSubtype(0)->getKind(),
+            liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, schemaTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, schemaSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, schemaMixed1) {
+  auto sharedPtrArrowSubsubtype0 = list(std::make_shared<Field>("a", large_utf8()));
+  auto sharedPtrArrowSubtype0 =
+      list(std::make_shared<Field>("a", sharedPtrArrowSubsubtype0));
+  auto sharedPtrArrowSubtype1 = list(std::make_shared<Field>("b", decimal(30, 4)));
+  auto sharedPtrArrowSubtype2 =
+      list(std::make_shared<Field>("c", timestamp(TimeUnit::type::MICRO)));
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", sharedPtrArrowSubtype0));
+  xFields.push_back(std::make_shared<Field>("b", sharedPtrArrowSubtype1));
+  xFields.push_back(std::make_shared<Field>("c", sharedPtrArrowSubtype2));
+  xFields.push_back(std::make_shared<Field>("d", boolean()));
+  xFields.push_back(std::make_shared<Field>("e", fixed_size_binary(5)));
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 5);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getFieldName(1), "b");
+  EXPECT_EQ(out->getFieldName(2), "c");
+  EXPECT_EQ(out->getFieldName(3), "d");
+  EXPECT_EQ(out->getFieldName(4), "e");
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(1)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(2)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(2)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(3)->getKind(), liborc::TypeKind::BOOLEAN);
+  EXPECT_EQ(out->getSubtype(4)->getKind(), liborc::TypeKind::BINARY);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(1)->getSubtype(0)->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getSubtype(2)->getSubtype(0)->getKind(), liborc::TypeKind::TIMESTAMP);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtype(0)->getKind(),
+            liborc::TypeKind::STRING);
+}
+
+// WriteORC tests
+// TEST(TestAdapterWriteNumerical, writeBoolEmpty0) {
+//   BooleanBuilder builder;
+//   std::shared_ptr<Array> array;
+//   (void)(builder.Finish(&array));
+//   std::shared_ptr<Table> table = std::make_shared<Table>({array},{std::String("a")});
+//   MemoryOutputStreamV2 file(DEFAULT_SMALL_MEM_STREAM_SIZE);
+//   std::unique_ptr<adapters::orc::ORCFileWriter>* writer;
+//   ORCFileWriter::Open(table->schema().get(),
+//                     const std::shared_ptr<io::FileOutputStream>& file,
+//                     std::shared_ptr<liborc::WriterOptions> options,
+//                     std::shared_ptr<ArrowWriterOptions> arrow_options,
+//                     std::unique_ptr<ORCFileWriter>* writer
+//                     )
+// }
+
+// Numeric
+
+// Bool
+TEST(TestAdapterWriteNumerical, writeBoolEmpty) {
+  BooleanBuilder builder;
+  std::shared_ptr<Array> array;
+  (void)(builder.Finish(&array));
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  ORC_UNIQUE_PTR<liborc::Type> schema(
+      liborc::Type::buildTypeFromString("struct<x:boolean>"));
+  liborc::WriterOptions options;
+  ORC_UNIQUE_PTR<liborc::Writer> writer = createWriter(*schema, &mem_stream, options);
+  uint64_t batchSize = 1024;
+  ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer->createRowBatch(batchSize);
+  liborc::StructVectorBatch* root =
+      internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+  liborc::LongVectorBatch* x =
+      internal::checked_cast<liborc::LongVectorBatch*>(root->fields[0]);
+  DataType* arrowType = boolean().get();
+  int64_t arrowOffset = 0;
+  int64_t orcOffset = 0;
+  Status st = adapters::orc::FillBatch(arrowType, x, arrowOffset, orcOffset, batchSize,
+                                       array.get());
+  if (!st.ok()) {
+    FAIL() << "ORC ColumnBatch not successfully filled";
+  }
+  EXPECT_EQ(x->numElements, 0);
+  EXPECT_FALSE(x->hasNulls);
+  EXPECT_EQ(arrowOffset, 0);
+  EXPECT_EQ(orcOffset, 0);
+  writer->add(*batch);
+  writer->close();
+}
+TEST(TestAdapterWriteNumerical, writeBoolNoNulls) {
+  BooleanBuilder builder;

Review comment:
       Thanks! We don't really have small tests with nonzero num of rows any more so this doesn't really apply.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772748



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);

Review comment:
       It has been renamed and is used in struct arrays. I have already provided commentary.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-791209166


   @pitrou Yup I found your changes to Random 6 days ago in Arrow-11662. Things did break after that. Now I’m trying to figure out whether it is my ORC writer or Decimal128 generator that needs to be fixed.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561967537



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +146,35 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] schema of the Arrow table
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \param[out] writer the returned writer object
+  /// \return Status
+  static Status Open(const std::shared_ptr<Schema>& schema,

Review comment:
       Nowadays we would rather return `Result<std::unique_ptr<OrcFileWriter>>` and forego the out-pointer parameter.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");

Review comment:
       Is it ok if it's not an actual filename? Is it just used for error messages and such?

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {

Review comment:
       Generally, everything that's not exposed in a `.h` file should be put in the anonymous namespace, to minimize the size of exported symbols (and potentially open up some optimization opportunities for the compiler).
   
   Since we're doing this, can you also do the same change for the read side?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray);

Review comment:
       Same comments here.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);

Review comment:
       Can use `GetView` as well.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {
+      return st;
+    }
+    if (arrowChunkOffset < numBatch && orcOffset < length) {

Review comment:
       Why repeat this condition? It's the same as in the `while` above.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;

Review comment:
       This is making a copy of the vector. Is it necessary? Or can you just take a const-ref?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {

Review comment:
       Also, please use `checked_pointer_cast`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+ private:
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;

Review comment:
       Our naming scheme mandates `out_stream_` and `orc_schema_` (no camelcase for variable names, including class and struct members).

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);

Review comment:
       Please, no camelCase for variable names :-)

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(

Review comment:
       The method here is returning a `Status`, it shouldn't throw on error.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();

Review comment:
       `num_rows`

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {

Review comment:
       Use a const-ref here.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));

Review comment:
       Since this is returning a Status, you should use `RETURN_NOT_OK` here.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       This should go into the namespaces below and/or be made static (or anonymous).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Can you put all private stuff in the anonymous namespace?

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();

Review comment:
       I find it a bit weird that `WriteTable` implicitly closes the writer. I would expect a separate `Status Close()` method for that. What do you think?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];

Review comment:
       Isn't the batch supposed to have been cleared before calling this function?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>

Review comment:
       Preferably use CamelCase for template parameters (or, at worse, UPPERCASE)...

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));

Review comment:
       You're passing `batch_size` even though `num_rows` may be smaller.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);

Review comment:
       This is making a spurious copy. Please use `GetView` instead.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;

Review comment:
       I'm curious: is this necessary? Does the ORC API need it?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null

Review comment:
       Does ORC really need the terminating null here?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,

Review comment:
       * `arrow_offset` and `orc_offset`. Also, mutable references are generally prohibited. Instead, pass a pointer (which clearly indicates it's gonna be mutated).
   * Why `Array*`? You're not supposed to mutate the array... do you mean `const Array&`?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);

Review comment:
       What is `incomingMask`? Doesn't seem used, is it?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;

Review comment:
       According to the example in `https://orc.apache.org/docs/core-cpp.html`, `numElements` should be the number of rows in _this_ batch only. But `orcOffset` would be the number of rows from the beginning, no?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);

Review comment:
       Can use `GetView`... and then reuse the same (template) function as for `Binary` and `String`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"

Review comment:
       This inclusion shouldn't be necessary.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {

Review comment:
       All these `FillBatch` functions are repeating the same looping and null selection logic that could be factored out (for example in a template function, a class...). Especially since we may want to speed up this up some day.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;

Review comment:
       Why is this commented out?

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/table_builder.h"

Review comment:
       These inclusions shouldn't be necessary. You can include `arrow/type_fwd.h` for general declarations.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();

Review comment:
       Is it useful to shortcut here?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;

Review comment:
       You can use `util::optional` to avoid the `shared_ptr` allocation.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>

Review comment:
       This doesn't seem used in this `.h` file.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;

Review comment:
       This is the default and no need to be explicit.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;

Review comment:
       So you're doing this properly here and not in other fill functions?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));

Review comment:
       It would be more readable to use `arrow::util::Decimal128`, IMHO:
   ```c++
   const Decimal128 dec_value(array->GetValue(arrow_offset));
   batch->values[orcOffset] = liborc::Int128(dec_value.high_bits(), dec_value.low_bits());
   ```
   

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       `const DataType&`. The policy is: pass a reference if it's const, a pointer if it's non-const (and will be mutated by the callee).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));

Review comment:
       Please use `nullptr` not `NULLPTR`, except in `.h` files.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();

Review comment:
       You can use `num_fields()`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);

Review comment:
       Hmm... You could use `VisitTypeInline`, for example, to avoid the annoyance of writing all this. You can find an example here: https://github.com/apache/arrow/blob/master/cpp/src/arrow/array/validate.cc#L47

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {

Review comment:
       Just use `RETURN_NOT_OK`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {

Review comment:
       We would probably use `int` or `int64_t`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       Make this `const`? Also, I don't understand the comment. Why doesn't it matter?

##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -163,3 +163,15 @@ def test_orcfile_empty(datadir):
         ]))),
     ])
     assert table.schema == expected_schema
+
+
+def test_orcfile_readwrite():
+    from pyarrow import orc
+    buffer_output_stream = pa.BufferOutputStream()
+    a = pa.array([1, None, 3, None])
+    b = pa.array([None, 'Arrow', None, 'ORC'])
+    table = pa.table({"int64": a, "utf8": b})
+    orc.write_table(buffer_output_stream, table)
+    buffer_reader = pa.BufferReader(buffer_output_stream.getvalue())
+    outputTable = orc.ORCFile(buffer_reader).read()
+    assert table.equals(outputTable)

Review comment:
       `output_table`, as per [PEP 8](https://www.python.org/dev/peps/pep-0008/)

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray);

Review comment:
       By the way, the `AppendBatch` vs. `FillBatch` naming is confusing. Can we make the difference explicit? One is writing to ORC, the other is reading from ORC.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +950,137 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  Type::type kind = type.id();
+  switch (kind) {
+    case Type::type::NA: {  // Makes out NULLPTR
+      out->reset();
+      break;
+    }
+    case Type::type::BOOL:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+      break;
+    case Type::type::INT8:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+      break;
+    case Type::type::INT16:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+      break;
+    case Type::type::INT32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::INT);
+      break;
+    case Type::type::INT64:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::LONG);
+      break;
+    case Type::type::FLOAT:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+      break;
+    case Type::type::DOUBLE:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+      break;
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case Type::type::STRING:
+    case Type::type::LARGE_STRING:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::STRING);
+      break;
+    case Type::type::BINARY:
+    case Type::type::LARGE_BINARY:
+    case Type::type::FIXED_SIZE_BINARY:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+      break;
+    case Type::type::DATE32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DATE);
+      break;
+    case Type::type::DATE64:
+    case Type::type::TIMESTAMP:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+      break;
+    case Type::type::DECIMAL128: {
+      const uint64_t precision =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).scale());
+      *out = liborc::createDecimalType(precision, scale);
+      break;
+    }
+    case Type::type::LIST:
+    case Type::type::FIXED_SIZE_LIST:
+    case Type::type::LARGE_LIST: {
+      std::shared_ptr<DataType> arrowChildType =
+          static_cast<const BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+      RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+      *out = liborc::createListType(std::move(orcSubtype));
+      break;
+    }
+    case Type::type::STRUCT: {
+      *out = liborc::createStructType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {
+        std::string fieldName = (*it)->name();
+        std::shared_ptr<DataType> arrowChildType = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+        RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+        (*out)->addStructField(fieldName, std::move(orcSubtype));
+      }
+      break;
+    }
+    case Type::type::MAP: {
+      std::shared_ptr<DataType> keyArrowType =
+          checked_cast<const MapType&>(type).key_type();
+      std::shared_ptr<DataType> itemArrowType =
+          checked_cast<const MapType&>(type).item_type();
+      ORC_UNIQUE_PTR<liborc::Type> keyORCType, itemORCType;
+      RETURN_NOT_OK(GetORCType(*keyArrowType, &keyORCType));
+      RETURN_NOT_OK(GetORCType(*itemArrowType, &itemORCType));
+      *out = liborc::createMapType(std::move(keyORCType), std::move(itemORCType));
+      break;
+    }
+    case Type::type::DENSE_UNION:
+    case Type::type::SPARSE_UNION: {
+      *out = liborc::createUnionType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const UnionType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {

Review comment:
       Same as above...

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,
+                         const int maxSize = DEFAULT_SMALL_MEM_STREAM_SIZE) {
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(
+          static_cast<liborc::OutputStream*>(new MemoryOutputStream(maxSize)));
+  ARROW_EXPECT_OK(writer->Open(tableIn->schema(), out_stream));
+  ARROW_EXPECT_OK(writer->Write(tableIn));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));

Review comment:
       For concision, you can write: `auto in_stream = std::make_shared<io::BufferReader>(...)`

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,
+                         const int maxSize = DEFAULT_SMALL_MEM_STREAM_SIZE) {
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(
+          static_cast<liborc::OutputStream*>(new MemoryOutputStream(maxSize)));

Review comment:
       What is `MemoryOutputStream`? You should use Arrow's `BufferOutputStream`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +950,137 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  Type::type kind = type.id();
+  switch (kind) {
+    case Type::type::NA: {  // Makes out NULLPTR
+      out->reset();
+      break;
+    }
+    case Type::type::BOOL:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+      break;
+    case Type::type::INT8:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+      break;
+    case Type::type::INT16:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+      break;
+    case Type::type::INT32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::INT);
+      break;
+    case Type::type::INT64:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::LONG);
+      break;
+    case Type::type::FLOAT:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+      break;
+    case Type::type::DOUBLE:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+      break;
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case Type::type::STRING:
+    case Type::type::LARGE_STRING:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::STRING);
+      break;
+    case Type::type::BINARY:
+    case Type::type::LARGE_BINARY:
+    case Type::type::FIXED_SIZE_BINARY:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+      break;
+    case Type::type::DATE32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DATE);
+      break;
+    case Type::type::DATE64:
+    case Type::type::TIMESTAMP:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+      break;
+    case Type::type::DECIMAL128: {
+      const uint64_t precision =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).scale());
+      *out = liborc::createDecimalType(precision, scale);
+      break;
+    }
+    case Type::type::LIST:
+    case Type::type::FIXED_SIZE_LIST:
+    case Type::type::LARGE_LIST: {
+      std::shared_ptr<DataType> arrowChildType =
+          static_cast<const BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+      RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+      *out = liborc::createListType(std::move(orcSubtype));
+      break;
+    }
+    case Type::type::STRUCT: {
+      *out = liborc::createStructType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {

Review comment:
       Please rephrase this as:
   ```c++
   for (const auto& field : checked_cast<const StructType&>(type).fields()) {
     ...
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);

Review comment:
       Similarly, return `Result<ORC_UNIQUE_PTR<liborc::Type>>`

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(

Review comment:
       This is already in `ORCWriter`, why are you re-writing this?

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,

Review comment:
       Our convention is "expected", not "predicted" :-)

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,35 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/api.h"

Review comment:
       `api.h` files are much too heavyweight, please make the inclusions more fine-grained.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table));
+}
+TEST(TestAdapterWriteGeneral, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(36, 2)),
+                                              field("decimal128z", decimal(31, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(36, 2)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(31, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin[2], string_[13];
+  std::string str;
+  for (int64_t i = 0; i < numRows / 2; i++) {
+    bin[0] = i % 128;
+    bin[1] = bin[0];
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(i % 128));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                        ->Append(Decimal128(std::to_string(i) + ".56")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                        ->Append(Decimal128(std::to_string(i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                        ->Append(INT64_C(1605547718999999999) + i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin, 2));
+  }
+  for (int64_t i = numRows / 2; i < numRows; i++) {
+    bin[0] = i % 256;
+    bin[1] = (i / 256) % 256;
+    str = "Arrow " + std::to_string(3 - 4 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(-(i % 128)));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(4 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                        ->Append(Decimal128(std::to_string(-i) + ".00")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                        ->Append(Decimal128(std::to_string(-i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                        ->Append(INT64_C(1605557718999999999) - i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin, 2));
+  }

Review comment:
       I hope you can find a way to cut down on all this test setup. For example, `arrow/testing/random.h` provides utilities to generate random array data.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);

Review comment:
       `auto schema = ::arrow::schema(fields);`

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       There doesn't seem to be much of a difference, though. The main one seems to be that the null terminator is appended for string data (why is that?).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {
+      return st;
+    }
+    if (arrowChunkOffset < numBatch && orcOffset < length) {
+      arrowIndexOffset = 0;
+      arrowChunkOffset++;
+    }
+  }
+  return Status::OK();
+}
+
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
-  // When subselecting fields on read, liborc will set some nodes to nullptr,
-  // so we need to check for nullptr before progressing
-  if (type == nullptr) {
+  // When subselecting fields on read, liborc will set some nodes to NULLPTR,
+  // so we need to check for NULLPTR before progressing
+  if (type == NULLPTR) {

Review comment:
       Can you undo the `NULLPTR` changes?

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));

Review comment:
       You can probably use `ArrayFromJSON` and `ChunkedArrayFromJSON` to make such test setup much terser.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       It's not about fixing it, it's about undoing unnecessary changes in this PR ;-)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r564213532



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);

Review comment:
       Done! Thanks!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773156



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray);

Review comment:
       I have renamed all the `FillBatch` `WriteBatch`. As for `AppendBatch` they currently remain since `ReadBatch` already exists and has a different usage.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] xhochy commented on pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
xhochy commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-755967695


   I can have a look on Monday / Tuesday.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614156557



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,44 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/array.h"
+#include "arrow/buffer.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/compute/cast.h"

Review comment:
       It actually does. Arrow has a lot more types than ORC hence Arrow2ORC(ORC2Arrow(x)) may not be the same as x. As a result we have to have casting for testing purposes.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567590991



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null

Review comment:
       The ORC library will delete all of these after the fact?  This actually looks like a memory leak looking at:
   https://github.com/apache/orc/blob/0128f817b0ab28fa2d0660737234ac966f0f5c50/c%2B%2B/src/MemoryPool.cc#L137 and https://github.com/apache/orc/blob/0128f817b0ab28fa2d0660737234ac966f0f5c50/c%2B%2B/include/orc/Vector.hh#L122
   
   If this is only temporary.  a zero copy way of doing this would be to assign data to the base pointer of the data vector + the offset of the current array element.  This would still probably be cheaper even if after writing you had to null out all values again




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-820129581






-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792154086


   @pitrou I have found the cause of the problem which is me failing to account for the possibility of orc::Decimal64VectorBatch. I will edit my writer to account for that.
   
   After this PR I'd like to work with you on adapter/testing/random. As you said before tests need to be added. Moreover we should expand random things to RecordBatches, ChunkedArrays & Tables as well as all types.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990299



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +453,80 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ORCFileWriter::Impl {
+ public:
+  Status Open(Schema* schema, const std::shared_ptr<io::FileOutputStream>& file,
+              std::shared_ptr<liborc::WriterOptions> options,
+              std::shared_ptr<ArrowWriterOptions> arrow_options,

Review comment:
       Thanks! I have changed that. Note that `liborc::WriterOptions` need to be wrapped in ArrowWriterOptions which will be done in the next ORC writer/reader release.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614508964



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {

Review comment:
       Thanks! Good catch. Since arrays get normalized before they are passed in this is no longer useful.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567617342



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       OK looking at ORC it seems that batch is just a temporary cache for passing data into the writer. in that case all this affects is temporary memory used.  Adding a comment to this affect would be useful.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612990896



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,

Review comment:
       @emkornfield @lidavidm Since I use the fact that real DATE64 and TIMESTAMP (with UNIT not equals NANO) can be cast to TIMESTAMP (using NANO) without getting beyond int64_t (because ORC essentially only supports NANO, see TimestampVectorBatch in https://orc.apache.org/docs/core-cpp.html) I don't think `arrow::random::RandomArrayGenerator.ArrayOf` can be used.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614511596



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));

Review comment:
       Thanks! Do you have recommendations? List and Map types are not frequently used but they do get used. I'd better fix this.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567592948



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||

Review comment:
       use a combination of BitBlockCounter and BitRunReader here.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream a pointer to the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream* output_stream);
+
+  /// \brief Write a table
+  ///
+  /// \param[in] table the Arrow table from which data is extracted
+  /// \return Status
+  Status Write(const Table& table);
+
+  /// \brief Close a file

Review comment:
       It closes the `std::unique_ptr<liborc::Writer> writer_` so yes closure does take place. However it doesn’t close the output stream. Doc clarified.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772081



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612933793



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -44,20 +45,13 @@
 #include "arrow/util/macros.h"
 #include "arrow/util/range.h"
 #include "arrow/util/visibility.h"
-
 #include "orc/Exceptions.hh"
-#include "orc/OrcFile.hh"
+
+constexpr uint64_t kOrcWriterBatchSize = 100000;

Review comment:
       Thanks! Yes these two constants have to be uint64_t due to the ORC APIs.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-764050107


   > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Sorry it didn’t appear in the right place but there is nothing parquet-related in this PR either. Can the parquet label be removed? Thanks!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567593501



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondNanos, 1);
+      }
+    }
+    case arrow::Type::type::DECIMAL:

Review comment:
       This should now be DECIMAL128 if DECIMAL256 is not going to be supported.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614598009



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       @emkornfield @pitrou It does actually look like `IOError` in this case. In the ORC reader `liborc::ParseError` is converted into `IOError` for IO errors and `Invalid` for other errors. Since this PR does not include fixing all issues in the ORC reader (which I would like to do in a latter PR after my current issues have been addressed) and that in the ORC writer there is only one instance of `liborc::ParseError` my current fix is adding the other two catches. However if you guys insist I can revamp error processing in the ORC reader tomorrow.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595076



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];

Review comment:
       Yes, they are. The batches are reused which is why it is necessary to overwrite everything.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       Ah. Yes. Eventually I plan to relocate the code to the place where we generate random arrays since this functionality helps generating random ChunkedArrays.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594464546



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       In newer Arrow code we use Result instead of Status & an output parameter.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r560634894



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Thanks! These functions are actually not that similar though.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r560636285



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),

Review comment:
       These tests are mostly amalgamated from the 31,000+ lines of ORC test that used to exist. Overall if we generate random test data we should be able to generate random Arrow tables and use them in ORC tests as well.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);

Review comment:
       Thanks! Definitely.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567580234



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    while (num_rows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        RETURN_NOT_OK(adapters::orc::WriteBatch(
+            (root->fields)[i], &(arrow_index_offset[i]), &(arrow_chunk_offset[i]),
+            batch_size, *(table.column(i))));
+      }
+      root->numElements = (root->fields)[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      num_rows -= batch_size;
+    }
+    return Status::OK();
+  }
+  Status Close() {
+    writer_->close();
+    return Status::OK();
+  }
+
+ private:
+  std::unique_ptr<liborc::Writer> writer_;
+  std::unique_ptr<liborc::OutputStream> out_stream_;
+};
+
+ORCFileWriter::~ORCFileWriter() {}
+
+ORCFileWriter::ORCFileWriter() { impl_.reset(new ORCFileWriter::Impl()); }
+
+Result<std::unique_ptr<ORCFileWriter>> ORCFileWriter::Open(
+    io::OutputStream& output_stream) {
+  std::unique_ptr<ORCFileWriter> result =
+      std::unique_ptr<ORCFileWriter>(new ORCFileWriter());
+  Status status = result->impl_->Open(output_stream);
+  if (status.ok()) {

Review comment:
       ```
   RETURN_NOT_OK(status);
   return result;
   ```
   is more idiomatic.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561405645



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);

Review comment:
       > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Thanks! There isn’t anything Parquet-related either. Can that be removed as well?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Thanks! That's in the ORC reader though. I can file a different PR to fix the reader (and add Union support to it).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Thanks! This one is in the ORC reader though. I can file a different PR to fix the reader (and add Union support to it).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;

Review comment:
       Thanks! I will fix all of them.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();

Review comment:
       Sure! 

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;

Review comment:
       It is. If the root->numElements is not set it is 0 and nothing gets written.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));

Review comment:
       This is intentional. Multiple ORC batches and multiple Arrow chunks are expected.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/table_builder.h"

Review comment:
       Thanks! This has been fixed.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"

Review comment:
       Thanks! This has been fixed.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       Sure! I will!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614499015



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612963356



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       Sure. Docs Added.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream a pointer to the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream* output_stream);
+
+  /// \brief Write a table
+  ///
+  /// \param[in] table the Arrow table from which data is extracted
+  /// \return Status
+  Status Write(const Table& table);
+
+  /// \brief Close a file

Review comment:
       It closes the `std::unique_ptr<liborc::Writer> writer_` so yes closure does take place. Hold on..




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612955262



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(kOrcWriterBatchSize);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    while (num_rows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        RETURN_NOT_OK(adapters::orc::WriteBatch(
+            (root->fields)[i], &(arrow_index_offset[i]), &(arrow_chunk_offset[i]),
+            kOrcWriterBatchSize, *(table.column(i))));
+      }
+      root->numElements = (root->fields)[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      num_rows -= kOrcWriterBatchSize;
+    }
+    return Status::OK();
+  }
+  Status Close() {

Review comment:
       nit: whitespace




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594461270



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r575731092



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    while (num_rows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        RETURN_NOT_OK(adapters::orc::WriteBatch(
+            (root->fields)[i], &(arrow_index_offset[i]), &(arrow_chunk_offset[i]),
+            batch_size, *(table.column(i))));
+      }
+      root->numElements = (root->fields)[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      num_rows -= batch_size;
+    }
+    return Status::OK();
+  }
+  Status Close() {
+    writer_->close();
+    return Status::OK();
+  }
+
+ private:
+  std::unique_ptr<liborc::Writer> writer_;
+  std::unique_ptr<liborc::OutputStream> out_stream_;
+};
+
+ORCFileWriter::~ORCFileWriter() {}
+
+ORCFileWriter::ORCFileWriter() { impl_.reset(new ORCFileWriter::Impl()); }
+
+Result<std::unique_ptr<ORCFileWriter>> ORCFileWriter::Open(
+    io::OutputStream& output_stream) {
+  std::unique_ptr<ORCFileWriter> result =
+      std::unique_ptr<ORCFileWriter>(new ORCFileWriter());
+  Status status = result->impl_->Open(output_stream);
+  if (status.ok()) {

Review comment:
       Thanks! 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r574939867



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondNanos, 1);
+      }
+    }
+    case arrow::Type::type::DECIMAL:
+      return WriteDecimalBatch(column_vector_batch, arrow_offset, orc_offset, length,
+                               array, incoming_mask);
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(column_vector_batch, arrow_offset, orc_offset, length,
+                              array, incoming_mask);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(column_vector_batch, arrow_offset,
+                                              orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(column_vector_batch, arrow_offset, orc_offset, length, array,
+                           incoming_mask);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790587488


   It seems there was a crash on the "AMD64 Conda C++" CI build. Can you take a look?
   Note that you should be able to reproduce this locally using our docker-compose setup.
   One way to do this is to first install the `archery` utility: https://arrow.apache.org/docs/developers/archery.html
   and then run `archery docker run conda-cpp`.
   
   (or you may also directly run `docker-compose run --rm conda-cpp`)


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r563393105



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));

Review comment:
       Done! Thank! `TableFromJSON` is very useful for empty table tests!

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,

Review comment:
       Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612963356



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614157196



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,189 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+/// \brief Construct a random weak composition of a nonnegative integer
+/// i.e. a way of writing it as the sum of a sequence of n non-negative
+/// integers.
+///
+/// \param[in] n the number of integers in the weak composition
+/// \param[in] sum the integer of which a random weak composition is generated
+/// \param[out] out The generated weak composition
+template <typename T, typename U>
+void RandWeakComposition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,
+                                                            arrow::TimeUnit::type type,
+                                                            double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  switch (type) {
+    case arrow::TimeUnit::type::SECOND: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::SECOND),
+          rand.Int64(size, kSecondMin, kSecondMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MILLI: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MILLI),
+          rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MICRO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MICRO),
+          rand.Int64(size, kMicroMin, kMicroMax, null_probability));
+    }
+    case arrow::TimeUnit::type::NANO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::NANO),
+          rand.Int64(size, kNanoMin, kNanoMax, null_probability));
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow TimeUnit: ", type);
+    }
+  }
+}
+
+std::shared_ptr<ChunkedArray> GenerateRandomChunkedArray(
+    const std::shared_ptr<DataType>& data_type, int64_t size, int64_t min_num_chunks,
+    int64_t max_num_chunks, double null_probability) {

Review comment:
       Sure. This and weak composition do not seem to belong to the ORC adapter tests as they are a lot more general.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614522859



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {

Review comment:
       Ah. Nope. The batches are reused so explicit marking of NonNulls as NonNulls is necessary.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614516188



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();

Review comment:
       Yup. Fixed.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612964375



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(kOrcWriterBatchSize);

Review comment:
       It is an internal parameter of the ORC adapter. Currently I do not plan to make it configurable though in practice it should be large.




-- 
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-764050107


   > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Thanks! There is nothing parquet-related in this PR either. Can the parquet label be removed 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595572



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it

Review comment:
       Yes. The type is simply not allowed in the GetORCType function.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614510493



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];

Review comment:
       Please correct me if I'm wrong: I know that they do and unlike Arrow ORC is not supposed to accommodate this behavior since we prioritize saving space as opposed to guaranteeing O(1) access in ORC. This is why I deliberately use `batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];` to skip these elements that shouldn't make their way into the ORC elements array.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614516746



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -410,19 +1087,33 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
       std::vector<std::shared_ptr<Field>> fields;
       std::vector<int8_t> type_codes;
       for (int child = 0; child < subtype_count; ++child) {
-        std::shared_ptr<DataType> elemtype;
-        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elemtype));
-        fields.push_back(field("_union_" + std::to_string(child), elemtype));
+        std::shared_ptr<DataType> elem_type;
+        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elem_type));
+        fields.push_back(field("_union_" + std::to_string(child), elem_type));
         type_codes.push_back(static_cast<int8_t>(child));
       }
       *out = sparse_union(fields, type_codes);
       break;
     }
     default: {
-      return Status::Invalid("Unknown Orc type kind: ", kind);
+      return Status::Invalid("Unknown Orc type kind: ", type->toString());

Review comment:
       TypeError is used haha since if default is reached there is indeed some TypeError in ORC.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -410,19 +1087,33 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
       std::vector<std::shared_ptr<Field>> fields;
       std::vector<int8_t> type_codes;
       for (int child = 0; child < subtype_count; ++child) {
-        std::shared_ptr<DataType> elemtype;
-        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elemtype));
-        fields.push_back(field("_union_" + std::to_string(child), elemtype));
+        std::shared_ptr<DataType> elem_type;
+        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elem_type));
+        fields.push_back(field("_union_" + std::to_string(child), elem_type));
         type_codes.push_back(static_cast<int8_t>(child));
       }
       *out = sparse_union(fields, type_codes);
       break;
     }
     default: {
-      return Status::Invalid("Unknown Orc type kind: ", kind);
+      return Status::Invalid("Unknown Orc type kind: ", type->toString());
     }
   }
-  return Status::OK();
+  return arrow::Status::OK();
+}
+
+Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const Schema& schema) {
+  int numFields = schema.num_fields();
+  ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createStructType();
+  for (int i = 0; i < numFields; i++) {
+    std::shared_ptr<Field> field = schema.field(i);
+    std::string field_name = field->name();
+    std::shared_ptr<DataType> arrow_child_type = field->type();
+    ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+        ::GetORCType(*arrow_child_type).ValueOrDie();

Review comment:
       Yup.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614556529



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);

Review comment:
       Thanks! Fixed!




-- 
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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-791345192


   As you prefer, though I suppose it's only the ORC writer _tests_ that need to be fixed, right?


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990362



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,14 +19,38 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/util/visibility.h"
+#include "orc/OrcFile.hh"
+
+namespace liborc = orc;
+
+#define ORC_THROW_NOT_OK(s)                   \

Review comment:
       Thanks! According to Wes these macros aren't actually allowed in adapter.h since they require including headers from ORC hence they have been removed.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r595293956



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       @pitrou You’re welcome! Ah in half of the cases what happened is that I never addressed issues on this page. I’m doing some major refactorization to address Micah’s most demanding comments this week haha.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594782467



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -163,3 +163,15 @@ def test_orcfile_empty(datadir):
         ]))),
     ])
     assert table.schema == expected_schema
+
+
+def test_orcfile_readwrite():
+    from pyarrow import orc
+    buffer_output_stream = pa.BufferOutputStream()
+    a = pa.array([1, None, 3, None])
+    b = pa.array([None, 'Arrow', None, 'ORC'])
+    table = pa.table({"int64": a, "utf8": b})
+    orc.write_table(buffer_output_stream, table)
+    buffer_reader = pa.BufferReader(buffer_output_stream.getvalue())
+    outputTable = orc.ORCFile(buffer_reader).read()
+    assert table.equals(outputTable)

Review comment:
       Ah sorry I didn't see this one. Fixed. :)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591678190



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,103 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    constexpr uint64_t batch_size = 1;  // Doesn't matter what it is

Review comment:
       @pitrou Ah..sure. It was low because an old bug was easier to isolate when it was set to 1. I have just made it 64K.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614157196



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,189 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+/// \brief Construct a random weak composition of a nonnegative integer
+/// i.e. a way of writing it as the sum of a sequence of n non-negative
+/// integers.
+///
+/// \param[in] n the number of integers in the weak composition
+/// \param[in] sum the integer of which a random weak composition is generated
+/// \param[out] out The generated weak composition
+template <typename T, typename U>
+void RandWeakComposition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,
+                                                            arrow::TimeUnit::type type,
+                                                            double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  switch (type) {
+    case arrow::TimeUnit::type::SECOND: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::SECOND),
+          rand.Int64(size, kSecondMin, kSecondMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MILLI: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MILLI),
+          rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MICRO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MICRO),
+          rand.Int64(size, kMicroMin, kMicroMax, null_probability));
+    }
+    case arrow::TimeUnit::type::NANO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::NANO),
+          rand.Int64(size, kNanoMin, kNanoMax, null_probability));
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow TimeUnit: ", type);
+    }
+  }
+}
+
+std::shared_ptr<ChunkedArray> GenerateRandomChunkedArray(
+    const std::shared_ptr<DataType>& data_type, int64_t size, int64_t min_num_chunks,
+    int64_t max_num_chunks, double null_probability) {

Review comment:
       Sure. This and weak composition do not seem to belong to an ORC adapter.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792360387


   @kou Thanks! I think I have already made all the changes in GLib (more precisely the Ruby tests) I think I need to make. Not sure why adding maps to buildable.rb led to a complaint about data_type = builder.value_data_type though.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-759780885


   @nevi-me Thanks! That’s done! I’m waiting for the PR to be reviewed and merged..


----------------------------------------------------------------
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.

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



[GitHub] [arrow] alamb commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
alamb commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-763560743


   I didn't see anything rust related in this PR so I removed the Rust label


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567579191



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }

Review comment:
       should this be uint64_t?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567574092



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       constexpr is even better.  Also style convention for constants is kOrcNaturalWriteSize




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614561787



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,189 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+/// \brief Construct a random weak composition of a nonnegative integer
+/// i.e. a way of writing it as the sum of a sequence of n non-negative
+/// integers.
+///
+/// \param[in] n the number of integers in the weak composition
+/// \param[in] sum the integer of which a random weak composition is generated
+/// \param[out] out The generated weak composition
+template <typename T, typename U>
+void RandWeakComposition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,
+                                                            arrow::TimeUnit::type type,
+                                                            double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  switch (type) {
+    case arrow::TimeUnit::type::SECOND: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::SECOND),
+          rand.Int64(size, kSecondMin, kSecondMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MILLI: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MILLI),
+          rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MICRO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MICRO),
+          rand.Int64(size, kMicroMin, kMicroMax, null_probability));
+    }
+    case arrow::TimeUnit::type::NANO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::NANO),
+          rand.Int64(size, kNanoMin, kNanoMax, null_probability));
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow TimeUnit: ", type);
+    }
+  }
+}
+
+std::shared_ptr<ChunkedArray> GenerateRandomChunkedArray(
+    const std::shared_ptr<DataType>& data_type, int64_t size, int64_t min_num_chunks,
+    int64_t max_num_chunks, double null_probability) {

Review comment:
       Wait. This function itself actually contains ORC-specific code such as the requirement that Date64 and Timestamp scalars must not overflow when cast to Timestamp NANO. Unless this requirement is actually universal in which case we should change how random arrays are canonically generated for these types we shouldn't really leave some function so ORC-specific in `testing/random.h`.
   
   However `void RandWeakComposition(int64_t n, T sum, std::vector<U>* out)` indeed belongs to `testing/random.h`. I have moved that one.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614158494



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       We can keep them for now but eventually these tests need to be replaced by Arrow2ORC(ORC2Arrow) ones.




-- 
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562000125



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;

Review comment:
       According to the example in https://orc.apache.org/docs/core-cpp.html, `numElements` should be the number of rows in _this_ batch only. But `orcOffset` would be the number of rows from the beginning, no?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612976502



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(kOrcWriterBatchSize);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    while (num_rows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        RETURN_NOT_OK(adapters::orc::WriteBatch(
+            (root->fields)[i], &(arrow_index_offset[i]), &(arrow_chunk_offset[i]),
+            kOrcWriterBatchSize, *(table.column(i))));
+      }
+      root->numElements = (root->fields)[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      num_rows -= kOrcWriterBatchSize;
+    }
+    return Status::OK();
+  }
+  Status Close() {

Review comment:
       Sure. Fixed.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792195893


   @pitrou  Now the problem has been fixed and it is ready for another review.
   @kou I managed to install Arrow GLib as a developer but didn't manage to run the tests in Ruby due to the following error:
   /Users/karlkatzen/Documents/code/arrow-dev/arrow/c_glib/test/test-extension-data-type.rb:19:in `<class:TestExtensionDataType>': uninitialized constant Arrow::ExtensionArray (NameError)
   The functionality I changed is in ARROW-1117, namely I load ORC MAPs as Arrow MAPs instead of LISTs of STRUCTs. arrow/c_glib/test/test-orc-file-reader.rb is the only file where tests fail. Please help me with the C_Glib part. Thanks!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-789737670


   @mathyingzhou What is the status of this PR? Are you waiting for another review? (feel free to ping)


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594816142



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       Sure. Just fixed that one.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] xhochy commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
xhochy commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r535594787



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +453,80 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ORCFileWriter::Impl {
+ public:
+  Status Open(Schema* schema, const std::shared_ptr<io::FileOutputStream>& file,
+              std::shared_ptr<liborc::WriterOptions> options,
+              std::shared_ptr<ArrowWriterOptions> arrow_options,

Review comment:
       Pass in `shared_ptr` instances via reference.
   ```suggestion
                 const std::shared_ptr<liborc::WriterOptions>& options,
                 const std::shared_ptr<ArrowWriterOptions>& arrow_options,
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));

Review comment:
       Use `ARROW_EXPECT_OK` instead of silencing the warning using `(void)`.
   ```suggestion
     ARROW_EXPECT_OK(adapters::orc::GetORCType(type, &out));
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BOOLEAN);
+}
+TEST(TestAdapterWriteConverter, typeInt8) {
+  DataType* type = int8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BYTE);
+}
+TEST(TestAdapterWriteConverter, typeInt16) {
+  DataType* type = int16().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::SHORT);
+}
+TEST(TestAdapterWriteConverter, typeInt32) {
+  DataType* type = int32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeInt64) {
+  DataType* type = int64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LONG);
+}
+TEST(TestAdapterWriteConverter, typeFloat) {
+  DataType* type = float32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::FLOAT);
+}
+TEST(TestAdapterWriteConverter, typeDouble) {
+  DataType* type = float64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DOUBLE);
+}
+TEST(TestAdapterWriteConverter, typeString) {
+  DataType* type = utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeLargeString) {
+  DataType* type = large_utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeBinary) {
+  DataType* type = binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeLargeBinary) {
+  DataType* type = large_binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinary) {
+  DataType* type = fixed_size_binary(3).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinaryZero) {
+  DataType* type = fixed_size_binary(0).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeDate32) {
+  DataType* type = date32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, typeDate64) {
+  DataType* type = date64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampSecond) {
+  DataType* type = timestamp(TimeUnit::type::SECOND).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMilli) {
+  DataType* type = timestamp(TimeUnit::type::MILLI).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMicro) {
+  DataType* type = timestamp(TimeUnit::type::MICRO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampNano) {
+  DataType* type = timestamp(TimeUnit::type::NANO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeDecimal) {
+  DataType* type = decimal(32, 5).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getPrecision(), 32);
+  EXPECT_EQ(out->getScale(), 5);
+}
+TEST(TestAdapterWriteConverter, typeList) {
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeLargeList) {
+  auto sharedPtrArrowType = large_list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeList) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 3);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeListZero) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 0);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeStructTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, typeStructSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));

Review comment:
       This can be condensed to 
   ```suggestion
     std::vector<std::shared_ptr<Field>> xFields{arrow::field("a", utf8())};
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BOOLEAN);
+}
+TEST(TestAdapterWriteConverter, typeInt8) {
+  DataType* type = int8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BYTE);
+}
+TEST(TestAdapterWriteConverter, typeInt16) {
+  DataType* type = int16().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::SHORT);
+}
+TEST(TestAdapterWriteConverter, typeInt32) {
+  DataType* type = int32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeInt64) {
+  DataType* type = int64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LONG);
+}
+TEST(TestAdapterWriteConverter, typeFloat) {
+  DataType* type = float32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::FLOAT);
+}
+TEST(TestAdapterWriteConverter, typeDouble) {
+  DataType* type = float64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DOUBLE);
+}
+TEST(TestAdapterWriteConverter, typeString) {
+  DataType* type = utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeLargeString) {
+  DataType* type = large_utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeBinary) {
+  DataType* type = binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeLargeBinary) {
+  DataType* type = large_binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinary) {
+  DataType* type = fixed_size_binary(3).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinaryZero) {
+  DataType* type = fixed_size_binary(0).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeDate32) {
+  DataType* type = date32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, typeDate64) {
+  DataType* type = date64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampSecond) {
+  DataType* type = timestamp(TimeUnit::type::SECOND).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMilli) {
+  DataType* type = timestamp(TimeUnit::type::MILLI).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMicro) {
+  DataType* type = timestamp(TimeUnit::type::MICRO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampNano) {
+  DataType* type = timestamp(TimeUnit::type::NANO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeDecimal) {
+  DataType* type = decimal(32, 5).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getPrecision(), 32);
+  EXPECT_EQ(out->getScale(), 5);
+}
+TEST(TestAdapterWriteConverter, typeList) {
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeLargeList) {
+  auto sharedPtrArrowType = large_list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeList) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 3);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeListZero) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 0);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeStructTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, typeStructSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeStruct) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getFieldName(1), "b");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeMap) {
+  auto sharedPtrArrowType = map(utf8(), int32());
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::MAP);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnionTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnionSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeDenseUnion) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = dense_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnionTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnionSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeSparseUnion) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  xFields.push_back(std::make_shared<Field>("b", int32()));
+  auto sharedPtrArrowType = sparse_union(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::UNION);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfList) {
+  auto sharedPtrArrowSubtype = list(std::make_shared<Field>("a", int32()));
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfMap) {
+  auto sharedPtrArrowSubtype = map(utf8(), int32());
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 2);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::MAP);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeListOfStructOfLists) {
+  auto sharedPtrArrowSubsubtype0 = list(std::make_shared<Field>("a", int8()));
+  auto sharedPtrArrowSubsubtype1 = list(std::make_shared<Field>("b", float64()));
+  auto sharedPtrArrowSubsubtype2 = list(std::make_shared<Field>("c", date32()));
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", sharedPtrArrowSubsubtype0));
+  xFields.push_back(std::make_shared<Field>("b", sharedPtrArrowSubsubtype1));
+  xFields.push_back(std::make_shared<Field>("c", sharedPtrArrowSubsubtype2));
+  auto sharedPtrArrowSubtype = struct_(xFields);
+  auto sharedPtrArrowType = list(std::make_shared<Field>("x", sharedPtrArrowSubtype));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 3);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtype(0)->getKind(),
+            liborc::TypeKind::BYTE);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(1)->getSubtype(0)->getKind(),
+            liborc::TypeKind::DOUBLE);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(2)->getSubtype(0)->getKind(),
+            liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, schemaTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, schemaSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, schemaMixed1) {
+  auto sharedPtrArrowSubsubtype0 = list(std::make_shared<Field>("a", large_utf8()));
+  auto sharedPtrArrowSubtype0 =
+      list(std::make_shared<Field>("a", sharedPtrArrowSubsubtype0));
+  auto sharedPtrArrowSubtype1 = list(std::make_shared<Field>("b", decimal(30, 4)));
+  auto sharedPtrArrowSubtype2 =
+      list(std::make_shared<Field>("c", timestamp(TimeUnit::type::MICRO)));
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", sharedPtrArrowSubtype0));
+  xFields.push_back(std::make_shared<Field>("b", sharedPtrArrowSubtype1));
+  xFields.push_back(std::make_shared<Field>("c", sharedPtrArrowSubtype2));
+  xFields.push_back(std::make_shared<Field>("d", boolean()));
+  xFields.push_back(std::make_shared<Field>("e", fixed_size_binary(5)));
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(sharedPtrSchema.get(), &out));
+  EXPECT_EQ(out->getSubtypeCount(), 5);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+  EXPECT_EQ(out->getFieldName(0), "a");
+  EXPECT_EQ(out->getFieldName(1), "b");
+  EXPECT_EQ(out->getFieldName(2), "c");
+  EXPECT_EQ(out->getFieldName(3), "d");
+  EXPECT_EQ(out->getFieldName(4), "e");
+  EXPECT_EQ(out->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(1)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(1)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(2)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(2)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(3)->getKind(), liborc::TypeKind::BOOLEAN);
+  EXPECT_EQ(out->getSubtype(4)->getKind(), liborc::TypeKind::BINARY);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(1)->getSubtype(0)->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getSubtype(2)->getSubtype(0)->getKind(), liborc::TypeKind::TIMESTAMP);
+  EXPECT_EQ(out->getSubtype(0)->getSubtype(0)->getSubtype(0)->getKind(),
+            liborc::TypeKind::STRING);
+}
+
+// WriteORC tests
+// TEST(TestAdapterWriteNumerical, writeBoolEmpty0) {
+//   BooleanBuilder builder;
+//   std::shared_ptr<Array> array;
+//   (void)(builder.Finish(&array));
+//   std::shared_ptr<Table> table = std::make_shared<Table>({array},{std::String("a")});
+//   MemoryOutputStreamV2 file(DEFAULT_SMALL_MEM_STREAM_SIZE);
+//   std::unique_ptr<adapters::orc::ORCFileWriter>* writer;
+//   ORCFileWriter::Open(table->schema().get(),
+//                     const std::shared_ptr<io::FileOutputStream>& file,
+//                     std::shared_ptr<liborc::WriterOptions> options,
+//                     std::shared_ptr<ArrowWriterOptions> arrow_options,
+//                     std::unique_ptr<ORCFileWriter>* writer
+//                     )
+// }
+
+// Numeric
+
+// Bool
+TEST(TestAdapterWriteNumerical, writeBoolEmpty) {
+  BooleanBuilder builder;
+  std::shared_ptr<Array> array;
+  (void)(builder.Finish(&array));
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  ORC_UNIQUE_PTR<liborc::Type> schema(
+      liborc::Type::buildTypeFromString("struct<x:boolean>"));
+  liborc::WriterOptions options;
+  ORC_UNIQUE_PTR<liborc::Writer> writer = createWriter(*schema, &mem_stream, options);
+  uint64_t batchSize = 1024;
+  ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer->createRowBatch(batchSize);
+  liborc::StructVectorBatch* root =
+      internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+  liborc::LongVectorBatch* x =
+      internal::checked_cast<liborc::LongVectorBatch*>(root->fields[0]);
+  DataType* arrowType = boolean().get();
+  int64_t arrowOffset = 0;
+  int64_t orcOffset = 0;
+  Status st = adapters::orc::FillBatch(arrowType, x, arrowOffset, orcOffset, batchSize,
+                                       array.get());
+  if (!st.ok()) {
+    FAIL() << "ORC ColumnBatch not successfully filled";
+  }
+  EXPECT_EQ(x->numElements, 0);
+  EXPECT_FALSE(x->hasNulls);
+  EXPECT_EQ(arrowOffset, 0);
+  EXPECT_EQ(orcOffset, 0);
+  writer->add(*batch);
+  writer->close();
+}
+TEST(TestAdapterWriteNumerical, writeBoolNoNulls) {
+  BooleanBuilder builder;

Review comment:
       Instead of using a builder, use `ArrayFromJSON`, e.g. `ArrayFromJSON(arrow::boolean(), "[true, false]")`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,14 +19,38 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/util/visibility.h"
+#include "orc/OrcFile.hh"
+
+namespace liborc = orc;
+
+#define ORC_THROW_NOT_OK(s)                   \

Review comment:
       If we have these macros in headers, we should call them `ARROW_ ORC_THROW_NOT_OK`.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r615643643



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       @mathyingzhou Fair enough.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612929323



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -44,20 +45,13 @@
 #include "arrow/util/macros.h"
 #include "arrow/util/range.h"
 #include "arrow/util/visibility.h"
-
 #include "orc/Exceptions.hh"
-#include "orc/OrcFile.hh"
+
+constexpr uint64_t kOrcWriterBatchSize = 100000;

Review comment:
       nit: power of 2 probably makes sense.  comment on how this was chosen (or if it was abitrary.  Also generally we avoid unsigned integers so if unsigned is required for API pursposes comment on it.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-775537466


   > @mathyingzhou Feel free to ping me when this is ready for review again!
   
   Thanks! Right now it seems that I will have to fix a bug in arrow/testing/random first if I want to further shorten the tests. I really don’t want to make this important PR dependent on these things though.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614513426



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+          GetORCType(*arrow_child_type).ValueOrDie();
+      return liborc::createListType(std::move(orc_subtype));
+    }
+    case arrow::Type::type::STRUCT: {
+      ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createStructType();
+      std::vector<std::shared_ptr<arrow::Field>> arrow_fields =
+          checked_cast<const arrow::StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<arrow::Field>>::iterator it = arrow_fields.begin();
+           it != arrow_fields.end(); ++it) {
+        std::string field_name = (*it)->name();
+        std::shared_ptr<arrow::DataType> arrow_child_type = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+            GetORCType(*arrow_child_type).ValueOrDie();
+        out_type->addStructField(field_name, std::move(orc_subtype));
+      }
+      return out_type;
+    }
+    case arrow::Type::type::MAP: {
+      std::shared_ptr<arrow::DataType> key_arrow_type =
+          checked_cast<const arrow::MapType&>(type).key_type();
+      std::shared_ptr<arrow::DataType> item_arrow_type =
+          checked_cast<const arrow::MapType&>(type).item_type();
+      ORC_UNIQUE_PTR<liborc::Type> key_orc_type =
+                                       GetORCType(*key_arrow_type).ValueOrDie(),
+                                   item_orc_type =
+                                       GetORCType(*item_arrow_type).ValueOrDie();
+      return liborc::createMapType(std::move(key_orc_type), std::move(item_orc_type));
+    }
+    case arrow::Type::type::DENSE_UNION:
+    case arrow::Type::type::SPARSE_UNION: {
+      ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createUnionType();
+      std::vector<std::shared_ptr<arrow::Field>> arrow_fields =
+          checked_cast<const arrow::UnionType&>(type).fields();
+      for (std::vector<std::shared_ptr<arrow::Field>>::iterator it = arrow_fields.begin();
+           it != arrow_fields.end(); ++it) {
+        std::string field_name = (*it)->name();
+        std::shared_ptr<arrow::DataType> arrow_child_type = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+            GetORCType(*arrow_child_type).ValueOrDie();
+        out_type->addUnionChild(std::move(orc_subtype));
+      }
+      return out_type;
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",

Review comment:
       Thanks! Switched to `Status::NotImplemented` since we don't have dictionary support in this PR but it will be supported.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612954380



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);

Review comment:
       nit: CreateWriter or if this is an orc symbol, having a namespace attached like orc::createWriter()




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-759104480


   @xhochy Please review it when you can. Thanks!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792387596


   @kou  Really thanks! I didn't realize that arrow::Type::type::MAP was not fully supported!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565771651



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();

Review comment:
       Thanks! All similar name issues have been fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567592354



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;

Review comment:
       as noted above you should probably use Buffer here and copy the bitmap buffer appropriately from the underlying Array, then pass that through so you can very quickly update current nullability.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772141



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990410



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));

Review comment:
       Thanks! All instances of `void` has been replaced by `ARROW_EXPECT_OK`. The code has also become a lot more compact.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r560636760



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +907,139 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType* type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  // Check for NULLPTR
+  if (type == NULLPTR) {

Review comment:
       This does need to be removed. We don’t need to add a feature to write NA type Arrow columns into 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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-799469048


   @mathyingzhou I still see a bunch of comments that are unanswered in https://github.com/apache/arrow/pull/8648/files , did you forget to push some changes?


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567581513



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       make this static constexpr.  Why doesn't it matter?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512073



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];

Review comment:
       Same comment as the list type: ORC does not tolerate Nulls occupying nonzero length in key and element batches of `orc::MapVectorBatch`.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594783196



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,17 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
-                   int64_t offset, int64_t length, ArrayBuilder* builder);
+                   int64_t offset, int64_t length, arrow::ArrayBuilder* builder);
+
+Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,

Review comment:
       @pitrou @emkornfield 




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561520435



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Thanks! That's in the ORC reader though. I can file a different PR to fix the reader (and add Union support to it).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Thanks! This one is in the ORC reader though. I can file a different PR to fix the reader (and add Union support to it).




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567592508



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {

Review comment:
       typically we don't use implicit booleanness of numbers.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562267448



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       Sure! I will!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-777997864


   Looks like in order to implementt ARROW-11117 I need to make some Ruby changes to the C Glib 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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792360387


   @kou Thanks! I think I have already made all the changes in GLib (more precisely the Ruby tests) I think I need to make. Not sure why adding MAPs to buildable.rb led to a complaint about data_type = builder.value_data_type though.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] kou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-778001836


   I can help it.
   Please ping me when C++ part is ready to merge. I'll adjust GLib and Ruby part to new behavior.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-757507160


   I have finished the Python binding as well. Note that I have made no changes to the Rust code.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567583851



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,

Review comment:
       please document parameters.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614499856



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);

Review comment:
       Thanks! 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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-777997864


   Looks like in order to implement ARROW-11117 I need to make some Ruby changes to the C Glib 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.

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



[GitHub] [arrow] codecov-io commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-757506115


   # [Codecov](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=h1) Report
   > Merging [#8648](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=desc) (552bf93) into [master](https://codecov.io/gh/apache/arrow/commit/1f32ca138d606c844da3e80ef139d21714a45592?el=desc) (1f32ca1) will **decrease** coverage by `0.00%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/arrow/pull/8648/graphs/tree.svg?width=650&height=150&src=pr&token=LpTCFbqVT1)](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #8648      +/-   ##
   ==========================================
   - Coverage   81.80%   81.80%   -0.01%     
   ==========================================
     Files         214      214              
     Lines       51383    51383              
   ==========================================
   - Hits        42034    42033       -1     
   - Misses       9349     9350       +1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [rust/arrow/src/buffer.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvYnVmZmVyLnJz) | `97.72% <ø> (ø)` | |
   | [rust/arrow/src/compute/kernels/aggregate.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL2FnZ3JlZ2F0ZS5ycw==) | `75.00% <ø> (ø)` | |
   | [rust/arrow/src/compute/kernels/arithmetic.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL2FyaXRobWV0aWMucnM=) | `89.83% <ø> (ø)` | |
   | [rust/arrow/src/compute/kernels/comparison.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvY29tcHV0ZS9rZXJuZWxzL2NvbXBhcmlzb24ucnM=) | `95.91% <ø> (ø)` | |
   | [rust/arrow/src/datatypes.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvZGF0YXR5cGVzLnJz) | `78.75% <ø> (ø)` | |
   | [rust/arrow/src/util/bit\_util.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9hcnJvdy9zcmMvdXRpbC9iaXRfdXRpbC5ycw==) | `100.00% <ø> (ø)` | |
   | [rust/parquet/src/encodings/encoding.rs](https://codecov.io/gh/apache/arrow/pull/8648/diff?src=pr&el=tree#diff-cnVzdC9wYXJxdWV0L3NyYy9lbmNvZGluZ3MvZW5jb2RpbmcucnM=) | `95.24% <0.00%> (-0.20%)` | :arrow_down: |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=footer). Last update [1f32ca1...552bf93](https://codecov.io/gh/apache/arrow/pull/8648?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567584931



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||

Review comment:
       This can be optimized quite a bit.  Anding the two masks together (see util/bit*) and then using BitBlockCounter or BitRunReader so that in the common cases these can be set in batches.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567578452



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +197,1930 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// Trivial
+TEST(TestAdapterWriteTrivial, writeZeroRowsNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {R"([])"});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeZeroRowsWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {R"([])"}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {R"([])"});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+
+// General
+TEST(TestAdapterWriteGeneral, writeAllNullsNew) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = schema(table_fields);
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  for (int i = 0; i < numCols; i++) {
+    for (int j = 0; j < 5; j++) {
+      int row_count = j % 2 ? 0 : num_rows / 2;
+      arrays[i][j] = rand.ArrayOf(table_fields[i]->type(), row_count, 1);
+    }
+  }
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+TEST(TestAdapterWriteGeneral, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(36, 2)),
+      field("decimal128z", decimal(31, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(36, 2)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(31, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin[2], string_[13];
+  std::string str;
+  for (int64_t i = 0; i < num_rows / 2; i++) {
+    bin[0] = i % 128;
+    bin[1] = bin[0];
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(i % 128));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                        ->Append(Decimal128(std::to_string(i) + ".56")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                        ->Append(Decimal128(std::to_string(i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                        ->Append(INT64_C(1605547718999999999) + i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin, 2));
+  }
+  for (int64_t i = num_rows / 2; i < num_rows; i++) {
+    bin[0] = i % 256;
+    bin[1] = (i / 256) % 256;
+    str = "Arrow " + std::to_string(3 - 4 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(-(i % 128)));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(4 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                        ->Append(Decimal128(std::to_string(-i) + ".00")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                        ->Append(Decimal128(std::to_string(-i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                        ->Append(INT64_C(1605557718999999999) - i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin, 2));
+  }
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table, 2 * kDefaultSmallMemStreamSize);
+}
+TEST(TestAdapterWriteGeneral, writeMixed) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(38, 6)),
+      field("decimal128z", decimal(38, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(38, 6)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(38, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+  char bin1[2], bin2[] = "", string_[12];
+  std::string str;
+  for (int64_t i = 0; i < num_rows / 2; i++) {
+    if (i % 2) {
+      bin1[0] = i % 256;
+      bin1[1] = (i - 1) % 256;
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int8Builder>(builders[1][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int32Builder>(builders[3][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Decimal128Builder>(builders[5][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                          ->Append(Decimal128(std::to_string(i))));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Date32Builder>(builders[7][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                          ->Append(INT64_C(1605548719999999999) + 2 * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders[9][1])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin1, 2));
+    } else {
+      str = "Arrow " + std::to_string(2 * i);
+      snprintf(string_, sizeof(string_), "%s", str.c_str());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BooleanBuilder>(builders[0][1])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(-(i % 128)));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int16Builder>(builders[2][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(-i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int64Builder>(builders[4][1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                          ->Append(Decimal128(std::to_string(i) + ".567890")));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Decimal128Builder>(builders[6][1])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(builders[8][1])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(builders[10][1])->AppendNull());
+    }
+  }
+  for (int64_t i = num_rows / 2; i < num_rows; i++) {
+    if (i % 2) {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int8Builder>(builders[1][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(-i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int32Builder>(builders[3][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Decimal128Builder>(builders[5][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                          ->Append(Decimal128(std::to_string(-i))));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Date32Builder>(builders[7][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                          ->Append(INT64_C(1606548719999999999) - INT64_C(29) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders[9][3])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin2, 0));
+    } else {
+      str = "Arrow " + std::to_string(2 * i);
+      snprintf(string_, sizeof(string_), "%s", str.c_str());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BooleanBuilder>(builders[0][3])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(i % 128));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int16Builder>(builders[2][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int64Builder>(builders[4][3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                          ->Append(Decimal128(std::to_string(-i) + ".543211")));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Decimal128Builder>(builders[6][3])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(builders[8][3])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(builders[10][3])->AppendNull());
+    }
+  }
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+
+// Float & Double
+// Equality might not work hence we do them separately here
+TEST(TestAdapterWriteFloat, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{field("float", float32()),
+                                                   field("double", float64())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  }
+
+  for (int i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+
+  std::shared_ptr<io::BufferOutputStream> buffer_output_stream =
+      io::BufferOutputStream::Create(kDefaultSmallMemStreamSize).ValueOrDie();
+  std::unique_ptr<adapters::orc::ORCFileWriter> writer =
+      adapters::orc::ORCFileWriter::Open(*buffer_output_stream).ValueOrDie();
+  ARROW_EXPECT_OK(writer->Write(*table));
+  ARROW_EXPECT_OK(writer->Close());
+  std::shared_ptr<Buffer> buffer = buffer_output_stream->Finish().ValueOrDie();
+  std::shared_ptr<io::RandomAccessFile> in_stream(new io::BufferReader(buffer));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> actual_output_table;
+  ARROW_EXPECT_OK(reader->Read(&actual_output_table));
+  EXPECT_EQ(actual_output_table->num_columns(), numCols);
+  EXPECT_EQ(actual_output_table->num_rows(), num_rows);
+  EXPECT_TRUE(actual_output_table->schema()->Equals(*(table->schema())));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(3)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(3)));
+}
+TEST(TestAdapterWriteFloat, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{field("float", float32()),
+                                                   field("double", float64())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  }
+
+  for (int i = 0; i < num_rows / 2; i++) {
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<FloatBuilder>(builders[0][1])->Append(i + 0.7));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<DoubleBuilder>(builders[1][1])->Append(-i + 0.43));
+  }
+  for (int i = num_rows / 2; i < num_rows; i++) {
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<FloatBuilder>(builders[0][3])->Append(2 * i - 2.12));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<DoubleBuilder>(builders[1][3])->Append(3 * i + 4.12));
+  }
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+
+  std::shared_ptr<io::BufferOutputStream> buffer_output_stream =
+      io::BufferOutputStream::Create(kDefaultSmallMemStreamSize).ValueOrDie();
+  std::unique_ptr<adapters::orc::ORCFileWriter> writer =
+      adapters::orc::ORCFileWriter::Open(*buffer_output_stream).ValueOrDie();
+  ARROW_EXPECT_OK(writer->Write(*table));
+  ARROW_EXPECT_OK(writer->Close());
+  std::shared_ptr<Buffer> buffer = buffer_output_stream->Finish().ValueOrDie();
+  std::shared_ptr<io::RandomAccessFile> in_stream(new io::BufferReader(buffer));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> actual_output_table;
+  ARROW_EXPECT_OK(reader->Read(&actual_output_table));
+  EXPECT_EQ(actual_output_table->num_columns(), numCols);
+  EXPECT_EQ(actual_output_table->num_rows(), num_rows);
+  EXPECT_TRUE(actual_output_table->schema()->Equals(*(table->schema())));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(3)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(3)));
+}
+TEST(TestAdapterWriteFloat, writeMixed) {
+  std::vector<std::shared_ptr<Field>> table_fields{field("float", float32()),
+                                                   field("double", float64())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  }
+
+  for (int i = 0; i < num_rows / 2; i++) {
+    if (i % 2) {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FloatBuilder>(builders[0][1])->Append(i + 0.7));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<DoubleBuilder>(builders[1][1])->AppendNull());
+    } else {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FloatBuilder>(builders[0][1])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<DoubleBuilder>(builders[1][1])->Append(-i + 0.43));
+    }
+  }
+  for (int i = num_rows / 2; i < num_rows; i++) {
+    if (i % 2) {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FloatBuilder>(builders[0][3])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<DoubleBuilder>(builders[1][3])->Append(3 * i + 4.12));
+    } else {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FloatBuilder>(builders[0][3])->Append(2 * i - 2.12));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<DoubleBuilder>(builders[1][3])->AppendNull());
+    }
+  }
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+
+  std::shared_ptr<io::BufferOutputStream> buffer_output_stream =
+      io::BufferOutputStream::Create(kDefaultSmallMemStreamSize).ValueOrDie();
+  std::unique_ptr<adapters::orc::ORCFileWriter> writer =
+      adapters::orc::ORCFileWriter::Open(*buffer_output_stream).ValueOrDie();
+  ARROW_EXPECT_OK(writer->Write(*table));
+  ARROW_EXPECT_OK(writer->Close());
+  std::shared_ptr<Buffer> buffer = buffer_output_stream->Finish().ValueOrDie();
+  std::shared_ptr<io::RandomAccessFile> in_stream(new io::BufferReader(buffer));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> actual_output_table;
+  ARROW_EXPECT_OK(reader->Read(&actual_output_table));
+  EXPECT_EQ(actual_output_table->num_columns(), numCols);
+  EXPECT_EQ(actual_output_table->num_rows(), num_rows);
+  EXPECT_TRUE(actual_output_table->schema()->Equals(*(table->schema())));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(0)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(0)->chunk(3)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(0, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(1)));
+  EXPECT_TRUE(actual_output_table->column(1)
+                  ->chunk(0)
+                  ->Slice(num_rows / 2, num_rows / 2)
+                  ->ApproxEquals(table->column(1)->chunk(3)));
+}
+
+// Converts
+// Since Arrow has way more types than ORC type conversions are unavoidable
+TEST(TestAdapterWriteConvert, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> input_fields{
+      field("date64", date64()),
+      field("ts0", timestamp(TimeUnit::SECOND)),
+      field("ts1", timestamp(TimeUnit::MILLI)),
+      field("ts2", timestamp(TimeUnit::MICRO)),
+      field("large_string", large_utf8()),
+      field("large_binary", large_binary()),
+      field("fixed_size_binary0", fixed_size_binary(0)),
+      field("fixed_size_binary", fixed_size_binary(5))},
+      output_fields{field("date64", timestamp(TimeUnit::NANO)),
+                    field("ts0", timestamp(TimeUnit::NANO)),
+                    field("ts1", timestamp(TimeUnit::NANO)),
+                    field("ts2", timestamp(TimeUnit::NANO)),
+                    field("large_string", utf8()),
+                    field("large_binary", binary()),
+                    field("fixed_size_binary0", binary()),
+                    field("fixed_size_binary", binary())};
+  std::shared_ptr<Schema> input_schema = std::make_shared<Schema>(input_fields),
+                          output_schema = std::make_shared<Schema>(output_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = input_fields.size();
+
+  ArrayBuilderMatrix buildersIn(numCols, ArrayBuilderVector(5, NULLPTR));
+  ArrayBuilderVector buildersOut(numCols, NULLPTR);
+
+  for (int i = 0; i < 5; i++) {
+    buildersIn[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date64Builder>());
+    buildersIn[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::SECOND), default_memory_pool()));
+    buildersIn[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MILLI), default_memory_pool()));
+    buildersIn[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MICRO), default_memory_pool()));
+    buildersIn[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeStringBuilder>());
+    buildersIn[5][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeBinaryBuilder>());
+    buildersIn[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(0)));
+    buildersIn[7][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(5)));
+  }
+
+  for (int col = 0; col < 4; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+  }
+  buildersOut[4] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  for (int col = 5; col < 8; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(buildersIn[col][chunk]->AppendNull());
+      ARROW_EXPECT_OK(buildersOut[col]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arraysIn(numCols, ArrayVector(5, NULLPTR));
+  ArrayVector arraysOut(numCols, NULLPTR);
+
+  ChunkedArrayVector cvIn, cvOut;
+  cvIn.reserve(numCols);
+  cvOut.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(buildersIn[col][i]->Finish(&arraysIn[col][i]));
+    }
+    ARROW_EXPECT_OK(buildersOut[col]->Finish(&arraysOut[col]));
+    cvIn.push_back(std::make_shared<ChunkedArray>(arraysIn[col]));
+    cvOut.push_back(std::make_shared<ChunkedArray>(arraysOut[col]));
+  }
+
+  std::shared_ptr<Table> input_table = Table::Make(input_schema, cvIn),
+                         expected_output_table = Table::Make(output_schema, cvOut);
+  AssertTableWriteReadEqual(input_table, expected_output_table);
+}
+TEST(TestAdapterWriteConvert, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> input_fields{
+      field("date64", date64()),
+      field("ts0", timestamp(TimeUnit::SECOND)),
+      field("ts1", timestamp(TimeUnit::MILLI)),
+      field("ts2", timestamp(TimeUnit::MICRO)),
+      field("large_string", large_utf8()),
+      field("large_binary", large_binary()),
+      field("fixed_size_binary0", fixed_size_binary(0)),
+      field("fixed_size_binary", fixed_size_binary(2))},
+      output_fields{field("date64", timestamp(TimeUnit::NANO)),
+                    field("ts0", timestamp(TimeUnit::NANO)),
+                    field("ts1", timestamp(TimeUnit::NANO)),
+                    field("ts2", timestamp(TimeUnit::NANO)),
+                    field("large_string", utf8()),
+                    field("large_binary", binary()),
+                    field("fixed_size_binary0", binary()),
+                    field("fixed_size_binary", binary())};
+  std::shared_ptr<Schema> input_schema = std::make_shared<Schema>(input_fields),
+                          output_schema = std::make_shared<Schema>(output_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = input_fields.size();
+
+  ArrayBuilderMatrix buildersIn(numCols, ArrayBuilderVector(5, NULLPTR));
+  ArrayBuilderVector buildersOut(numCols, NULLPTR);
+
+  for (int i = 0; i < 5; i++) {
+    buildersIn[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date64Builder>());
+    buildersIn[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::SECOND), default_memory_pool()));
+    buildersIn[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MILLI), default_memory_pool()));
+    buildersIn[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MICRO), default_memory_pool()));
+    buildersIn[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeStringBuilder>());
+    buildersIn[5][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeBinaryBuilder>());
+    buildersIn[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(0)));
+    buildersIn[7][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(2)));
+  }
+
+  for (int col = 0; col < 4; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+  }
+  buildersOut[4] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  for (int col = 5; col < 8; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin1[2], bin2[3], bin3[] = "", string_[12];
+  std::string str;
+
+  for (int64_t i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    bin1[0] = i % 256;
+    bin1[1] = (i + 1) % 256;
+    bin2[0] = (2 * i) % 256;
+    bin2[1] = (2 * i + 1) % 256;
+    bin2[2] = (i - 1) % 256;
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(std::static_pointer_cast<Date64Builder>(buildersIn[0][chunk])
+                        ->Append(INT64_C(1605758461555) + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[1][chunk])
+                        ->Append(INT64_C(1605758461) + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[2][chunk])
+                        ->Append(INT64_C(1605758461000) + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[3][chunk])
+                        ->Append(INT64_C(1605758461000111) + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<LargeStringBuilder>(buildersIn[4][chunk])
+                        ->Append(string_));
+    ARROW_EXPECT_OK(std::static_pointer_cast<LargeBinaryBuilder>(buildersIn[5][chunk])
+                        ->Append(bin2, 3));
+    ARROW_EXPECT_OK(std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[6][chunk])
+                        ->Append(bin3));
+    ARROW_EXPECT_OK(std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[7][chunk])
+                        ->Append(bin1));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[0])
+                        ->Append(INT64_C(1605758461555000000) + INT64_C(1000000) * i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[1])
+                        ->Append(INT64_C(1605758461000000000) + INT64_C(1000000000) * i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[2])
+                        ->Append(INT64_C(1605758461000000000) + INT64_C(1000000) * i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[3])
+                        ->Append(INT64_C(1605758461000111000) + INT64_C(1000) * i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(buildersOut[4])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(buildersOut[5])->Append(bin2, 3));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(buildersOut[6])->Append(bin3, 0));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(buildersOut[7])->Append(bin1, 2));
+  }
+
+  ArrayMatrix arraysIn(numCols, ArrayVector(5, NULLPTR));
+  ArrayVector arraysOut(numCols, NULLPTR);
+
+  ChunkedArrayVector cvIn, cvOut;
+  cvIn.reserve(numCols);
+  cvOut.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(buildersIn[col][i]->Finish(&arraysIn[col][i]));
+    }
+    ARROW_EXPECT_OK(buildersOut[col]->Finish(&arraysOut[col]));
+    cvIn.push_back(std::make_shared<ChunkedArray>(arraysIn[col]));
+    cvOut.push_back(std::make_shared<ChunkedArray>(arraysOut[col]));
+  }
+
+  std::shared_ptr<Table> input_table = Table::Make(input_schema, cvIn),
+                         expected_output_table = Table::Make(output_schema, cvOut);
+  AssertTableWriteReadEqual(input_table, expected_output_table);
+}
+TEST(TestAdapterWriteConvert, writeMixed) {
+  std::vector<std::shared_ptr<Field>> input_fields{
+      field("date64", date64()),
+      field("ts0", timestamp(TimeUnit::SECOND)),
+      field("ts1", timestamp(TimeUnit::MILLI)),
+      field("ts2", timestamp(TimeUnit::MICRO)),
+      field("large_string", large_utf8()),
+      field("large_binary", large_binary()),
+      field("fixed_size_binary0", fixed_size_binary(0)),
+      field("fixed_size_binary", fixed_size_binary(3))},
+      output_fields{field("date64", timestamp(TimeUnit::NANO)),
+                    field("ts0", timestamp(TimeUnit::NANO)),
+                    field("ts1", timestamp(TimeUnit::NANO)),
+                    field("ts2", timestamp(TimeUnit::NANO)),
+                    field("large_string", utf8()),
+                    field("large_binary", binary()),
+                    field("fixed_size_binary0", binary()),
+                    field("fixed_size_binary", binary())};
+  std::shared_ptr<Schema> input_schema = std::make_shared<Schema>(input_fields),
+                          output_schema = std::make_shared<Schema>(output_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = input_fields.size();
+
+  ArrayBuilderMatrix buildersIn(numCols, ArrayBuilderVector(5, NULLPTR));
+  ArrayBuilderVector buildersOut(numCols, NULLPTR);
+
+  for (int i = 0; i < 5; i++) {
+    buildersIn[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date64Builder>());
+    buildersIn[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::SECOND), default_memory_pool()));
+    buildersIn[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MILLI), default_memory_pool()));
+    buildersIn[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::MICRO), default_memory_pool()));
+    buildersIn[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeStringBuilder>());
+    buildersIn[5][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<LargeBinaryBuilder>());
+    buildersIn[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(0)));
+    buildersIn[7][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<FixedSizeBinaryBuilder>(fixed_size_binary(3)));
+  }
+
+  for (int col = 0; col < 4; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+  }
+  buildersOut[4] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  for (int col = 5; col < 8; col++) {
+    buildersOut[col] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin1[3], bin2[4], bin3[] = "", string_[13];
+  std::string str;
+
+  for (int64_t i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    if (i % 2) {
+      str = "Arrow " + std::to_string(-4 * i + 8);
+      snprintf(string_, sizeof(string_), "%s", str.c_str());
+      ARROW_EXPECT_OK(std::static_pointer_cast<Date64Builder>(buildersIn[0][chunk])
+                          ->Append(INT64_C(1605758461555) + 3 * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersIn[1][chunk])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[2][chunk])
+                          ->Append(INT64_C(1605758461000) - 14 * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersIn[3][chunk])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<LargeStringBuilder>(buildersIn[4][chunk])
+                          ->Append(string_));
+      ARROW_EXPECT_OK(std::static_pointer_cast<LargeBinaryBuilder>(buildersIn[5][chunk])
+                          ->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[6][chunk])
+              ->Append(bin3));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[7][chunk])
+              ->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[0])
+                          ->Append(INT64_C(1605758461555000000) + INT64_C(3000000) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersOut[1])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[2])
+                          ->Append(INT64_C(1605758461000000000) - INT64_C(14000000) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersOut[3])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<StringBuilder>(buildersOut[4])
+                          ->Append("Arrow " + std::to_string(-4 * i + 8)));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[5])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[6])->Append(bin3, 0));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[7])->AppendNull());
+    } else {
+      bin1[0] = i % 256;
+      bin1[1] = (i + 1) % 256;
+      bin1[2] = (i - 1) % 256;
+      bin2[0] = (29 * i - 192) % 256;
+      bin2[1] = (2 * i + 1) % 256;
+      bin2[2] = (4 * i + 103) % 256;
+      bin2[3] = (17 * i + 122) % 256;
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Date64Builder>(buildersIn[0][chunk])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[1][chunk])
+                          ->Append(INT64_C(1605758461) + INT64_C(61) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersIn[2][chunk])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersIn[3][chunk])
+                          ->Append(INT64_C(1605758461000111) + INT64_C(1021) * i));
+      ARROW_EXPECT_OK(std::static_pointer_cast<LargeStringBuilder>(buildersIn[4][chunk])
+                          ->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<LargeBinaryBuilder>(buildersIn[5][chunk])
+                          ->Append(bin2, 4));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[6][chunk])
+              ->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<FixedSizeBinaryBuilder>(buildersIn[7][chunk])
+              ->Append(bin1));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersOut[0])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersOut[1])
+              ->Append(INT64_C(1605758461000000000) + INT64_C(61000000000) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<TimestampBuilder>(buildersOut[2])->AppendNull());
+      ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(buildersOut[3])
+                          ->Append(INT64_C(1605758461000111000) + INT64_C(1021000) * i));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(buildersOut[4])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[5])->Append(bin2, 4));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[6])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<BinaryBuilder>(buildersOut[7])->Append(bin1, 3));
+    }
+  }
+
+  ArrayMatrix arraysIn(numCols, ArrayVector(5, NULLPTR));
+  ArrayVector arraysOut(numCols, NULLPTR);
+
+  ChunkedArrayVector cvIn, cvOut;
+  cvIn.reserve(numCols);
+  cvOut.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(buildersIn[col][i]->Finish(&arraysIn[col][i]));
+    }
+    ARROW_EXPECT_OK(buildersOut[col]->Finish(&arraysOut[col]));
+    cvIn.push_back(std::make_shared<ChunkedArray>(arraysIn[col]));
+    cvOut.push_back(std::make_shared<ChunkedArray>(arraysOut[col]));
+  }
+
+  std::shared_ptr<Table> input_table = Table::Make(input_schema, cvIn),
+                         expected_output_table = Table::Make(output_schema, cvOut);
+  AssertTableWriteReadEqual(input_table, expected_output_table);
+}
+
+// Nested types
+TEST(TestAdapterWriteNested, writeMixedListStruct) {
+  std::vector<std::shared_ptr<Field>> table_fields0{
+      std::make_shared<Field>("a", utf8()), std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("struct", struct_(table_fields0)), field("list", list(int32()))};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+  auto sharedPtrArrowType0 = table_fields[0]->type();
+  auto sharedPtrArrowType1 = table_fields[1]->type();
+
+  int64_t num_rows = 10000;
+  int64_t numCols0 = table_fields0.size();
+
+  //#0 struct<a:string,b:int>
+  ArrayBuilderMatrix builders0(numCols0, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders0[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders0[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  }
+  std::string str;
+  char string_[10];
+  for (int i = 0; i < num_rows; i++) {
+    int chunk = i < (num_rows / 2) ? 1 : 3;
+    str = "Test " + std::to_string(i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    if (i % 2) {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders0[0][chunk])->Append(string_));
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int32Builder>(builders0[1][chunk])->AppendNull());
+    } else {
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<StringBuilder>(builders0[0][chunk])->AppendNull());
+      ARROW_EXPECT_OK(
+          std::static_pointer_cast<Int32Builder>(builders0[1][chunk])->Append(i));
+    }
+  }
+
+  int arrayBitmapSize = num_rows / 16;
+  uint8_t bitmaps0[2][arrayBitmapSize];
+  for (int i = 0; i < arrayBitmapSize; i++) {
+    for (int j = 0; j < 2; j++) {
+      bitmaps0[j][i] = 153;  // 10011001
+    }
+  }
+
+  std::vector<std::shared_ptr<BufferBuilder>> bufferBuilders0(
+      2, std::make_shared<BufferBuilder>());
+  std::vector<std::shared_ptr<Buffer>> bitmapBuffers0(2, NULLPTR);
+
+  for (int i = 0; i < 2; i++) {
+    ARROW_EXPECT_OK(bufferBuilders0[i]->Resize(arrayBitmapSize));
+    ARROW_EXPECT_OK(bufferBuilders0[i]->Append(bitmaps0[i], arrayBitmapSize));
+    ARROW_EXPECT_OK(bufferBuilders0[i]->Finish(&bitmapBuffers0[i]));
+  }
+
+  ArrayMatrix subarrays0(5, ArrayVector(numCols0, NULLPTR));
+  ArrayVector av0;
+  av0.reserve(5);
+
+  for (int i = 0; i < 5; i++) {
+    for (int col = 0; col < numCols0; col++) {
+      ARROW_EXPECT_OK(builders0[col][i]->Finish(&subarrays0[i][col]));
+    }
+    if (i == 1 || i == 3) {
+      av0.push_back(std::make_shared<StructArray>(
+          sharedPtrArrowType0, num_rows / 2, subarrays0[i], bitmapBuffers0[(i - 1) / 2]));
+    } else {
+      av0.push_back(std::make_shared<StructArray>(sharedPtrArrowType0, 0, subarrays0[i]));
+    }
+  }
+
+  std::shared_ptr<ChunkedArray> carray0 = std::make_shared<ChunkedArray>(av0);
+
+  //#1 List
+
+  ArrayBuilderVector valuesBuilders1(5, NULLPTR), offsetsBuilders1(3, NULLPTR);
+  ArrayVector valuesArrays1(5, NULLPTR), offsetsArrays1(3, NULLPTR);
+
+  for (int i = 0; i < 5; i++) {
+    valuesBuilders1[i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  }
+  for (int i = 0; i < 3; i++) {
+    offsetsBuilders1[i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  }
+  int arrayOffsetSize = num_rows / 2 + 1;
+  int32_t offsets1[2][arrayOffsetSize];
+
+  offsets1[0][0] = 0;
+  offsets1[1][0] = 0;
+  for (int i = 0; i < num_rows; i++) {
+    int offsetsChunk = i < (num_rows / 2) ? 0 : 1;
+    int valuesChunk = 2 * offsetsChunk + 1;
+    int offsetsOffset = offsetsChunk * num_rows / 2;
+    switch (i % 4) {
+      case 0: {
+        offsets1[offsetsChunk][i + 1 - offsetsOffset] =
+            offsets1[offsetsChunk][i - offsetsOffset];
+        break;
+      }
+      case 1: {
+        ARROW_EXPECT_OK(
+            std::static_pointer_cast<Int32Builder>(valuesBuilders1[valuesChunk])
+                ->Append(i - 1));
+        offsets1[offsetsChunk][i + 1 - offsetsOffset] =
+            offsets1[offsetsChunk][i - offsetsOffset] + 1;
+        break;
+      }
+      case 2: {
+        for (int j = 0; j < 8; j++) {
+          ARROW_EXPECT_OK(valuesBuilders1[valuesChunk]->AppendNull());
+        }
+        offsets1[offsetsChunk][i + 1 - offsetsOffset] =
+            offsets1[offsetsChunk][i - offsetsOffset] + 8;
+        break;
+      }
+      default: {
+        for (int j = 0; j < 3; j++) {
+          ARROW_EXPECT_OK(valuesBuilders1[valuesChunk]->AppendNull());
+        }
+        ARROW_EXPECT_OK(
+            std::static_pointer_cast<Int32Builder>(valuesBuilders1[valuesChunk])
+                ->Append(i - 1));
+        for (int j = 0; j < 3; j++) {
+          ARROW_EXPECT_OK(valuesBuilders1[valuesChunk]->AppendNull());
+        }
+        offsets1[offsetsChunk][i + 1 - offsetsOffset] =
+            offsets1[offsetsChunk][i - offsetsOffset] + 7;
+      }
+    }
+  }
+
+  uint8_t bitmaps1[2][arrayBitmapSize];
+  for (int i = 0; i < arrayBitmapSize; i++) {
+    for (int j = 0; j < 2; j++) {
+      bitmaps1[j][i] = 165;  // 10100101
+    }
+  }
+
+  BufferBuilderVector bitmapBufferBuilders1(2, std::make_shared<BufferBuilder>()),
+      offsetsBufferBuilders1(2, std::make_shared<BufferBuilder>());
+  BufferVector bitmapBuffers1(2, NULLPTR), offsetsBuffers1(2, NULLPTR);
+
+  int arrayOffsetSizeBytes = 4 * arrayOffsetSize;
+
+  for (int i = 0; i < 2; i++) {
+    ARROW_EXPECT_OK(bitmapBufferBuilders1[i]->Resize(arrayBitmapSize));
+    ARROW_EXPECT_OK(bitmapBufferBuilders1[i]->Append(bitmaps1[i], arrayBitmapSize));
+    ARROW_EXPECT_OK(bitmapBufferBuilders1[i]->Finish(&bitmapBuffers1[i]));
+    ARROW_EXPECT_OK(offsetsBufferBuilders1[i]->Resize(arrayOffsetSizeBytes));
+    ARROW_EXPECT_OK(offsetsBufferBuilders1[i]->Append(offsets1[i], arrayOffsetSizeBytes));
+    ARROW_EXPECT_OK(offsetsBufferBuilders1[i]->Finish(&offsetsBuffers1[i]));
+  }
+  for (int i = 0; i < 3; i++) {
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int32Builder>(offsetsBuilders1[i])->Append(0));
+    ARROW_EXPECT_OK(offsetsBuilders1[i]->Finish(&offsetsArrays1[i]));
+  }
+
+  ArrayVector av1;
+  av1.reserve(5);
+
+  for (int i = 0; i < 5; i++) {
+    ARROW_EXPECT_OK(valuesBuilders1[i]->Finish(&valuesArrays1[i]));
+    if (i == 1 || i == 3) {
+      av1.push_back(std::make_shared<ListArray>(
+          sharedPtrArrowType1, num_rows / 2, offsetsBuffers1[(i - 1) / 2],
+          valuesArrays1[i], bitmapBuffers1[(i - 1) / 2]));
+    } else {
+      av1.push_back(
+          ListArray::FromArrays(*offsetsArrays1[i / 2], *valuesArrays1[i]).ValueOrDie());
+    }
+  }
+
+  std::shared_ptr<ChunkedArray> carray1 = std::make_shared<ChunkedArray>(av1);
+
+  ChunkedArrayVector cv{carray0, carray1};
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+TEST(TestAdapterWriteNested, writeMixedConvert) {
+  std::vector<std::shared_ptr<Field>> input_fields{
+      field("large_list", large_list(int32())),
+      field("fixed_size_list", fixed_size_list(int32(), 3)),
+      field("map", map(int32(), int32()))},
+      output_fields{
+          field("large_list", list(int32())), field("fixed_size_list", list(int32())),
+          field("map", list(struct_({field("key", int32()), field("value", int32())})))};
+
+  std::shared_ptr<Schema> input_schema = std::make_shared<Schema>(input_fields);
+  std::shared_ptr<Schema> output_schema = std::make_shared<Schema>(output_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = input_fields.size();
+
+  ArrayBuilderVector valuesBuilders0In(5, NULLPTR), offsetsBuilders0In(3, NULLPTR),

Review comment:
       nullptr should be fine in .cc 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614523227



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;

Review comment:
       Ah. Thanks for the catch! I never considered it before..
   
   Shall we add a big-endian machine to the CICD pipeline? That way we can check for similar issues in the future.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-764050107


   > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Sorry it didn’t appear in the right place but there is nothing parquet-related in this PR either. Can the parquet label be removed? Thanks!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] xhochy commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
xhochy commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r560408479



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +471,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return 128 * 1024; }

Review comment:
       Where does this constant come from? Should this be stated as a constant somewhere?

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +471,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return 128 * 1024; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  int64_t get_length() { return length_; }

Review comment:
       ```suggestion
     int64_t length() { return length_; }
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;

Review comment:
       Always use `{..}`
   ```suggestion
     if (array->null_count() || incomingMask) {
         batch->hasNulls = true;
      }
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType* type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema* schema, ORC_UNIQUE_PTR<liborc::Type>* out);

Review comment:
       Input arguments should be by reference.
   ```suggestion
   Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
   
   Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType* type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema* schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);

Review comment:
       Arguments that are also written to should be passed in as pointers.
   ```suggestion
   Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
                    int64_t* arrowOffset, int64_t* orcOffset, int64_t length, Array* parray,
                    std::vector<bool>* incomingMask = NULLPTR);
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Please keep using `nullptr` in `cc` files, we only use the macro in headers.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +471,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return 128 * 1024; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  int64_t get_length() { return length_; }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::make_shared<liborc::WriterOptions>();
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(schema.get(), &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+ private:
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::shared_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;

Review comment:
       Can these be `std::unique_ptr`? You only seem to pass the raw pointer to ORC, not the unique one.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Please merge this with `FillBinaryBatch`, the only difference is `GetValue` vs  `GetString`. This should be templateable.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +471,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return 128 * 1024; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  int64_t get_length() { return length_; }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::make_shared<liborc::WriterOptions>();
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(schema.get(), &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }

Review comment:
       ```suggestion
     }
     
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),

Review comment:
       What is the reason that you use slightly different schemas in every test? Maybe you can generalize this to have a reusable test data generation.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {

Review comment:
       Please refactor this to have the switch outside of the look and e.g. have constants set.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       What is the `incomingMask` about? I don't see where it is used.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);

Review comment:
       The code from here until the end of the function seems to be the same for the tests above, please  factor this out into a utility function.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +907,139 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType* type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  // Check for NULLPTR
+  if (type == NULLPTR) {

Review comment:
       What's the reason that `NULLPTR` is an allowed input?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();

Review comment:
       Don't use `auto` where the type is not visible somewhere else in the 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512702



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",

Review comment:
       Thanks! Switched to `Status::NotImplemented` since we don't have dictionary or union support in this PR but they will be supported.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614603354



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       Restored.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567582052



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,17 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
-                   int64_t offset, int64_t length, ArrayBuilder* builder);
+                   int64_t offset, int64_t length, arrow::ArrayBuilder* builder);
+
+Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,

Review comment:
       Please add documentation here.   The style used in this code base is to order parameters in the order:
   1.  Input first,
   2. Input output next.
   3. Output variables.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790297527


   @pitrou  Yes now it is ready for another review. I have fixed all the issues you mentioned, found and fixed a previously hidden bug and shortened the tests to about 650 lines (with more tests!) It should be a lot neater this time.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773246



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;

Review comment:
       Ah I need to remove them. They turn out to be useless.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614510493



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];

Review comment:
       Please correct me if I'm wrong: I know that they do and unlike Arrow ORC is not supposed to accommodate this behavior since we prioritize saving space as opposed to guaranteeing O(1) access in ORC. This is why I deliberately use `batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];` to skip these elements that shouldn't make their way into the ORC elements array. Similarly we have to normalize struct arrays since in ORC children (fields) must be marked as Null if the parent (i.e. struct) is marked as Null. 




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594472371



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {
+      return st;
+    }
+    if (arrowChunkOffset < numBatch && orcOffset < length) {
+      arrowIndexOffset = 0;
+      arrowChunkOffset++;
+    }
+  }
+  return Status::OK();
+}
+
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
-  // When subselecting fields on read, liborc will set some nodes to nullptr,
-  // so we need to check for nullptr before progressing
-  if (type == nullptr) {
+  // When subselecting fields on read, liborc will set some nodes to NULLPTR,
+  // so we need to check for NULLPTR before progressing
+  if (type == NULLPTR) {

Review comment:
       I have




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594461039



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       I have fixed that.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612956096



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,

Review comment:
       @lidavidm did you recently check in code that could replace this?




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594462966



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");

Review comment:
       Yes. In fact this is really just to make sure all virtual methods of the parent class are implemented.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-817451185


   @pitrou I completely revamped adapter_util.cc and switched to arrow::ArrayDataVisitor. That is, all comments have been addressed. Please 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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-773880147


   I will make some further changes to the code to address Micah Kornfield's concerns but pretty much that's it. Further simplification of the tests require serious expansion of arrow/testing/random and arrow/compute/cast to include nested types (and in the case of the latter at least the fixed_size_binary -> binary cast) or it will remain very ugly. If possible I can work on such features in at least 2 separate PRs.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r595203890



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       Thanks, did you forget to push the changes? I also checked out your branch locally, in case the Github UI wasn't up-to-date.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] kou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792388622


   Sorry for that... :-)


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594489348



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       Indeed, however since `output_stream` is mutable it should be passed as `io::OutputStream* output_stream` :-)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565771530



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790297527


   @pitrou  Yes now it is ready for another 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();

Review comment:
       Ah actually we don't need it for this release since we don't support NA or union types. I need to add it when implementing the ORC reader & writer for union types though. 




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567578953



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table));
+}
+TEST(TestAdapterWriteGeneral, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(36, 2)),
+                                              field("decimal128z", decimal(31, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(36, 2)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(31, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin[2], string_[13];
+  std::string str;
+  for (int64_t i = 0; i < numRows / 2; i++) {
+    bin[0] = i % 128;
+    bin[1] = bin[0];
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(i % 128));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                        ->Append(Decimal128(std::to_string(i) + ".56")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                        ->Append(Decimal128(std::to_string(i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                        ->Append(INT64_C(1605547718999999999) + i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin, 2));
+  }
+  for (int64_t i = numRows / 2; i < numRows; i++) {
+    bin[0] = i % 256;
+    bin[1] = (i / 256) % 256;
+    str = "Arrow " + std::to_string(3 - 4 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(-(i % 128)));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(4 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                        ->Append(Decimal128(std::to_string(-i) + ".00")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                        ->Append(Decimal128(std::to_string(-i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                        ->Append(INT64_C(1605557718999999999) - i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin, 2));
+  }

Review comment:
       I would recommend looking through the Arrow Parquet 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612933592



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +71,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       Thanks! Fixed.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567575377



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       Although the style guide (Google's C++ style guide) has been updated to recommend required output parameter the prevaling style in Arrow's code base is to pass output parameters by pointer.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       Ah. Yes. Eventually I plan to relocate the code to the place where we generate random arrays since this functionality helps generating random ChunkedArrays. 




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772435



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>

Review comment:
       It is actually used in `adapter.cc` and has been moved there.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();

Review comment:
       Ah actually we don't need it for this release. I need to add it when implementing the ORC reader & writer for union types though. 




-- 
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614023111



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream a pointer to the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream* output_stream);
+
+  /// \brief Write a table
+  ///
+  /// \param[in] table the Arrow table from which data is extracted
+  /// \return Status
+  Status Write(const Table& table);
+
+  /// \brief Close a file

Review comment:
       Does it actually close the file (i.e. the output stream)? It doesn't seem to. Can you make the docstring less ambiguous?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
       Can you keep these namespaces? We would like to avoid accidentally exposing symbols in the root namespace.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {

Review comment:
       The `normalized` argument isn't used anywhere, can you remove it?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -410,19 +1087,33 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
       std::vector<std::shared_ptr<Field>> fields;
       std::vector<int8_t> type_codes;
       for (int child = 0; child < subtype_count; ++child) {
-        std::shared_ptr<DataType> elemtype;
-        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elemtype));
-        fields.push_back(field("_union_" + std::to_string(child), elemtype));
+        std::shared_ptr<DataType> elem_type;
+        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elem_type));
+        fields.push_back(field("_union_" + std::to_string(child), elem_type));
         type_codes.push_back(static_cast<int8_t>(child));
       }
       *out = sparse_union(fields, type_codes);
       break;
     }
     default: {
-      return Status::Invalid("Unknown Orc type kind: ", kind);
+      return Status::Invalid("Unknown Orc type kind: ", type->toString());

Review comment:
       `TypeError` or `NotImplemented` here

##########
File path: python/pyarrow/_orc.pyx
##########
@@ -109,3 +113,28 @@ cdef class ORCReader(_Weakrefable):
                 check_status(deref(self.reader).Read(indices, &sp_table))
 
         return pyarrow_wrap_table(sp_table)
+
+cdef class ORCWriter(_Weakrefable):
+    cdef:
+        object source
+        unique_ptr[ORCFileWriter] writer
+
+    def open(self, object source):
+        cdef:
+            shared_ptr[COutputStream] rd_handle
+        self.source = source
+        get_writer(source, &rd_handle)
+        with nogil:
+            self.writer = move(GetResultValue[unique_ptr[ORCFileWriter]](
+                ORCFileWriter.Open(rd_handle.get())))

Review comment:
       `ORCFileWriter` doesn't keep a strong reference to the `shared_ptr[COutputStream]`, which is a local variable. This means the stream can be destroyed when this function exists.
   
   You should store the `shared_ptr[COutputStream]` either in the Python `ORCWriter` object, or in the C++ `ORCFileWriter` object. The latter sounds better to me.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       Two things:
   1) we should catch all possible exception classes, not one of them (suggested mapping: `NotImplementedYet` => `Status::NotImplemented`, `InvalidArgument` => `Status::Invalid`, `ParseError` => `Status::Invalid`)
   2) it would be worthwhile factoring the exception catching out, see e.g. https://github.com/apache/arrow/blob/master/cpp/src/parquet/exception.h#L34-L43
   

##########
File path: cpp/src/arrow/testing/random.h
##########
@@ -310,6 +310,16 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::shared_ptr<Array> FixedSizeBinary(int64_t size, int32_t byte_width,
                                          double null_probability = 0);
 
+  // /// \brief Generate a random StructArray
+  // ///
+  // /// \param[in] children Vector of Arrays containing the data for each child
+  // /// \param[in] size The size of the generated list array
+  // /// \param[in] null_probability the probability of a list value being null
+  // ///
+  // /// \return a generated Array
+  // std::shared_ptr<Array> Struct(const ArrayVector& children, int64_t size,
+  //                               double null_probability);
+

Review comment:
       If this is not implemented in this PR, can you remove the commented declaration?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow

Review comment:
       You don't have to close those namespaces before opening the anonymous namespace!

##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       I don't understand why the tests against example files were removed here. I think it would be worthwhile to keep them, especially if there's no new tests to replace them.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {
+namespace {
 
-using internal::checked_cast;
+using arrow::internal::checked_cast;
 
-// The number of nanoseconds in a second
-constexpr int64_t kOneSecondNanos = 1000000000LL;
-
-Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
-                         int64_t offset, int64_t length, ArrayBuilder* abuilder) {
-  auto builder = checked_cast<StructBuilder*>(abuilder);
-  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+arrow::Status AppendStructBatch(const liborc::Type* type,

Review comment:
       Adding `arrow::` everywhere seems like a gratuitous change.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);

Review comment:
       `checked_cast` or `checked_pointer_cast`

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -44,20 +45,15 @@
 #include "arrow/util/macros.h"
 #include "arrow/util/range.h"
 #include "arrow/util/visibility.h"
-
 #include "orc/Exceptions.hh"
-#include "orc/OrcFile.hh"
+
+// The following are required by ORC to be uint64_t
+constexpr uint64_t kOrcWriterBatchSize = 128 * 1024;
+constexpr uint64_t kOrcNaturalWriteSize = 128 * 1024;

Review comment:
       `static constexpr`: no need to expose them as library symbols.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;

Review comment:
       Put those in `arrow::adapters::orc` and/or make them static.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {

Review comment:
       Instead of enumerating all those explicitly, use a `default:` clause?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();

Review comment:
       Should probably add this for types without a validity bitmap:
   ```c++
   if (!internal::HasValidityBitmap(child->type_id())) {
     new_children[i] = NormalizeArray(child);
     continue;
   }
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;

Review comment:
       This won't work on big-endian machines. Possible solution (untested):
   ```c++
   const Decimal128 dec_value(array.GetValue(running_arrow_offset));
   batch->values[running_orc_offset] = static_cast<int64_t>(dec_value.low_bits());
   ```
   

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);

Review comment:
       Also `checked_pointer_cast` here and below.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);

Review comment:
       This won't work on big-endian machines. Instead, create a Decimal128 to grab the low and high bits.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(

Review comment:
       Simpler:
   ```c++
   auto new_child_array_data = child->data()->Copy();
   new_child_array_data->buffers[0] = std::move(final_child_bitmap);
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {

Review comment:
       This loop that fills the nulls array should only run if `array.null_count()` is not 0, right?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];

Review comment:
       You must also take into account `list_array.value_offset()` just like in the non-null case. Some list arrays might have null entries with a non-zero length.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];

Review comment:
       Same comment as above for list type: need to take into account the value offsets here.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));

Review comment:
       For the record, calling `Slice` for each list element will be costly. No need to fix this now, but in case you care about performance...

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,44 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/array.h"
+#include "arrow/buffer.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/compute/cast.h"

Review comment:
       Is the compute module needed to test ORC functionality? I'm a bit surprised.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+          GetORCType(*arrow_child_type).ValueOrDie();

Review comment:
       Please don't use `ValueOrDie` unless errors should never happen in practice. Instead:
   ```c++
   ARROW_ASSIGN_OR_RAISE(auto orc_child_type, GetOrcType(*arrow_child_type));
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",

Review comment:
       Please raise `Status::TypeError` or `Status::NotImplemented` here.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+          GetORCType(*arrow_child_type).ValueOrDie();
+      return liborc::createListType(std::move(orc_subtype));
+    }
+    case arrow::Type::type::STRUCT: {
+      ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createStructType();
+      std::vector<std::shared_ptr<arrow::Field>> arrow_fields =
+          checked_cast<const arrow::StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<arrow::Field>>::iterator it = arrow_fields.begin();
+           it != arrow_fields.end(); ++it) {
+        std::string field_name = (*it)->name();
+        std::shared_ptr<arrow::DataType> arrow_child_type = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+            GetORCType(*arrow_child_type).ValueOrDie();
+        out_type->addStructField(field_name, std::move(orc_subtype));
+      }
+      return out_type;
+    }
+    case arrow::Type::type::MAP: {
+      std::shared_ptr<arrow::DataType> key_arrow_type =
+          checked_cast<const arrow::MapType&>(type).key_type();
+      std::shared_ptr<arrow::DataType> item_arrow_type =
+          checked_cast<const arrow::MapType&>(type).item_type();
+      ORC_UNIQUE_PTR<liborc::Type> key_orc_type =
+                                       GetORCType(*key_arrow_type).ValueOrDie(),
+                                   item_orc_type =
+                                       GetORCType(*item_arrow_type).ValueOrDie();
+      return liborc::createMapType(std::move(key_orc_type), std::move(item_orc_type));
+    }
+    case arrow::Type::type::DENSE_UNION:
+    case arrow::Type::type::SPARSE_UNION: {
+      ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createUnionType();
+      std::vector<std::shared_ptr<arrow::Field>> arrow_fields =
+          checked_cast<const arrow::UnionType&>(type).fields();
+      for (std::vector<std::shared_ptr<arrow::Field>>::iterator it = arrow_fields.begin();
+           it != arrow_fields.end(); ++it) {
+        std::string field_name = (*it)->name();
+        std::shared_ptr<arrow::DataType> arrow_child_type = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+            GetORCType(*arrow_child_type).ValueOrDie();
+        out_type->addUnionChild(std::move(orc_subtype));
+      }
+      return out_type;
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",

Review comment:
       Here as well, either `Status::TypeError` or `Status::NotImplemented`

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {

Review comment:
       Hmm... you said "done" but I see no change here. Am I missing something?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -410,19 +1087,33 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
       std::vector<std::shared_ptr<Field>> fields;
       std::vector<int8_t> type_codes;
       for (int child = 0; child < subtype_count; ++child) {
-        std::shared_ptr<DataType> elemtype;
-        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elemtype));
-        fields.push_back(field("_union_" + std::to_string(child), elemtype));
+        std::shared_ptr<DataType> elem_type;
+        RETURN_NOT_OK(GetArrowType(type->getSubtype(child), &elem_type));
+        fields.push_back(field("_union_" + std::to_string(child), elem_type));
         type_codes.push_back(static_cast<int8_t>(child));
       }
       *out = sparse_union(fields, type_codes);
       break;
     }
     default: {
-      return Status::Invalid("Unknown Orc type kind: ", kind);
+      return Status::Invalid("Unknown Orc type kind: ", type->toString());
     }
   }
-  return Status::OK();
+  return arrow::Status::OK();
+}
+
+Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const Schema& schema) {
+  int numFields = schema.num_fields();
+  ORC_UNIQUE_PTR<liborc::Type> out_type = liborc::createStructType();
+  for (int i = 0; i < numFields; i++) {
+    std::shared_ptr<Field> field = schema.field(i);
+    std::string field_name = field->name();
+    std::shared_ptr<DataType> arrow_child_type = field->type();
+    ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+        ::GetORCType(*arrow_child_type).ValueOrDie();

Review comment:
       `ARROW_ASSIGN_OR_RAISE`, or similar

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());

Review comment:
       `checked_cast<const Decimal128Type&>`

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,189 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+/// \brief Construct a random weak composition of a nonnegative integer
+/// i.e. a way of writing it as the sum of a sequence of n non-negative
+/// integers.
+///
+/// \param[in] n the number of integers in the weak composition
+/// \param[in] sum the integer of which a random weak composition is generated
+/// \param[out] out The generated weak composition
+template <typename T, typename U>
+void RandWeakComposition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,
+                                                            arrow::TimeUnit::type type,
+                                                            double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  switch (type) {
+    case arrow::TimeUnit::type::SECOND: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::SECOND),
+          rand.Int64(size, kSecondMin, kSecondMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MILLI: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MILLI),
+          rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MICRO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MICRO),
+          rand.Int64(size, kMicroMin, kMicroMax, null_probability));
+    }
+    case arrow::TimeUnit::type::NANO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::NANO),
+          rand.Int64(size, kNanoMin, kNanoMax, null_probability));
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow TimeUnit: ", type);
+    }
+  }
+}
+
+std::shared_ptr<ChunkedArray> GenerateRandomChunkedArray(
+    const std::shared_ptr<DataType>& data_type, int64_t size, int64_t min_num_chunks,
+    int64_t max_num_chunks, double null_probability) {

Review comment:
       I suggest factoring this functionality in `testing/random.h` under the form:
   ```c++
   class ARROW_TESTING_EXPORT RandomArrayGenerator {
     // [snip]
     std::shared_ptr<ChunkedArray> Chunked(const std::shared_ptr<Array>& Array, int num_chunks);
   }
   ```
   

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();

Review comment:
       `checked_cast`...

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");

Review comment:
       Ok, then can you add a comment mentioning this? This would help readers of the code :-)

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,17 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
-                   int64_t offset, int64_t length, ArrayBuilder* builder);
+                   int64_t offset, int64_t length, arrow::ArrayBuilder* builder);
+
+Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,

Review comment:
       No need for a formal docstring, but you could still add a comment explaining what this does. Especially the `arrow_index_offset`, `arrow_chunk_offset` and `length` parameters.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();

Review comment:
       `checked_cast` or `checked_pointer_cast`




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594889437



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {

Review comment:
       Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r575733146



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       Thanks! This has been fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614743037



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       @pitrou Fair enough. Well, back then in Dec and Jan I did manual tests using the ORC adapter to write ORC files and then loaded them using pyorc and compared the results. So we should be good.
   
   Of course in the future I can add some more tests (read using Arrow, write without Arrow and vice versa). Can we get this PR out there though? The functionality has been very stable since early Jan and the amount of bugs affecting the actual ORC files we have caught during the past 3 months is 2-4.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614523227



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;

Review comment:
       Ah. Thanks for the catch! I never considered it before..




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614747421



##########
File path: python/pyarrow/_orc.pyx
##########
@@ -109,3 +113,28 @@ cdef class ORCReader(_Weakrefable):
                 check_status(deref(self.reader).Read(indices, &sp_table))
 
         return pyarrow_wrap_table(sp_table)
+
+cdef class ORCWriter(_Weakrefable):
+    cdef:
+        object source
+        unique_ptr[ORCFileWriter] writer
+
+    def open(self, object source):
+        cdef:
+            shared_ptr[COutputStream] rd_handle
+        self.source = source
+        get_writer(source, &rd_handle)
+        with nogil:
+            self.writer = move(GetResultValue[unique_ptr[ORCFileWriter]](
+                ORCFileWriter.Open(rd_handle.get())))

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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790619759


   That said, it's just a matter of implementing said generation properly :-)


----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612955728



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       some docs here would be good.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       nit: RandIntPartition?




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567611241



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null

Review comment:
       It looks like ORC might off on the ASAN check build




----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou closed pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #8648:
URL: https://github.com/apache/arrow/pull/8648


   


-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594463095



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }

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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567602645



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       typically batch sizes have implications for memory and file layouts, so a better explanation for readers not familiar with ORC would be useful.  1024 is actually pretty small if only 1024 rows are stored by group.  Typically would want go by memory usage but that can be difficult so larger batch size could potentially be better.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792360387


   @kou Thanks! I think I have already made all the changes in GLib (more precisely the Ruby tests) I think I need to make. Not sure why adding MAPs to buildable.rb led to a complaint about data_type = builder.value_data_type though. If I understand it correctly value_data_type is actually the Type struct in type_fwd.h. Not sure why we get “cannot create instance of abstract (non-instantiatable) type ‘GArrowDataType’”.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512116



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();

Review comment:
       Thanks! 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream a pointer to the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream* output_stream);
+
+  /// \brief Write a table
+  ///
+  /// \param[in] table the Arrow table from which data is extracted
+  /// \return Status
+  Status Write(const Table& table);
+
+  /// \brief Close a file

Review comment:
       It closes the `std::unique_ptr<liborc::Writer> writer_` so yes closure does take place. Doc clarified.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614408484



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -44,20 +45,15 @@
 #include "arrow/util/macros.h"
 #include "arrow/util/range.h"
 #include "arrow/util/visibility.h"
-
 #include "orc/Exceptions.hh"
-#include "orc/OrcFile.hh"
+
+// The following are required by ORC to be uint64_t
+constexpr uint64_t kOrcWriterBatchSize = 128 * 1024;
+constexpr uint64_t kOrcNaturalWriteSize = 128 * 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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612954582



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       Is this the best error?  ParseError sounds like something other than IO or is catch all.




-- 
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.

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



[GitHub] [arrow] kou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
kou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792356850


   Sure. I take over the GLib part.
   I'll also reply about the `@rpath` related problem (the `uninitialized constant` error is caused by this) on the dev@ mailing list later.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614403349



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990362



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,14 +19,38 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
 #include "arrow/type.h"
 #include "arrow/util/visibility.h"
+#include "orc/OrcFile.hh"
+
+namespace liborc = orc;
+
+#define ORC_THROW_NOT_OK(s)                   \

Review comment:
       These macros aren't actually allowed in adapter.h hence they have been removed.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();

Review comment:
       Thanks! Ah actually we don't need it for this release since we don't support NA (ORC doesn't have it) or union types. I do need to add it when implementing the ORC reader & writer for union types though. 




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595951



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +197,1930 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// Trivial
+TEST(TestAdapterWriteTrivial, writeZeroRowsNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {R"([])"});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeZeroRowsWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {R"([])"}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {R"([])"});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+
+// General
+TEST(TestAdapterWriteGeneral, writeAllNullsNew) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = schema(table_fields);
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  for (int i = 0; i < numCols; i++) {
+    for (int j = 0; j < 5; j++) {
+      int row_count = j % 2 ? 0 : num_rows / 2;
+      arrays[i][j] = rand.ArrayOf(table_fields[i]->type(), row_count, 1);
+    }
+  }
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));

Review comment:
       i see it isn't supported in 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.

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



[GitHub] [arrow] github-actions[bot] commented on pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-725811236


   https://issues.apache.org/jira/browse/ARROW-7906


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614171622



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       Those are slightly different, though. Roundtripping between Arrow and ORC doesn't validate that ORC data is correct, or that we are able to read foreign-produced ORC 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567596202



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +197,1930 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// Trivial
+TEST(TestAdapterWriteTrivial, writeZeroRowsNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {R"([])"});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeZeroRowsWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {R"([])"}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {R"([])"});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+
+// General
+TEST(TestAdapterWriteGeneral, writeAllNullsNew) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = schema(table_fields);
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  for (int i = 0; i < numCols; i++) {
+    for (int j = 0; j < 5; j++) {
+      int row_count = j % 2 ? 0 : num_rows / 2;
+      arrays[i][j] = rand.ArrayOf(table_fields[i]->type(), row_count, 1);
+    }
+  }
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));

Review comment:
       Yup. It is not supported in ORC now so we can't. I can file an ORC ticket though.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567585355



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];

Review comment:
       why would the data be not null?  are these being reused?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567580521



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       this should clarify ownership semantics (and who needs to call close on output_stream




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614156557



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,44 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/array.h"
+#include "arrow/buffer.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/compute/cast.h"

Review comment:
       It actually does. Arrow has a lot more types than ORC hence Arrow2ORC(ORC2Arrow(x)) may not be the same as x.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r550990445



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +217,15960 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// Arrow2ORC type converter tests
+
+TEST(TestAdapterWriteConverter, typeBool) {
+  DataType* type = boolean().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BOOLEAN);
+}
+TEST(TestAdapterWriteConverter, typeInt8) {
+  DataType* type = int8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BYTE);
+}
+TEST(TestAdapterWriteConverter, typeInt16) {
+  DataType* type = int16().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::SHORT);
+}
+TEST(TestAdapterWriteConverter, typeInt32) {
+  DataType* type = int32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeInt64) {
+  DataType* type = int64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LONG);
+}
+TEST(TestAdapterWriteConverter, typeFloat) {
+  DataType* type = float32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::FLOAT);
+}
+TEST(TestAdapterWriteConverter, typeDouble) {
+  DataType* type = float64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DOUBLE);
+}
+TEST(TestAdapterWriteConverter, typeString) {
+  DataType* type = utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeLargeString) {
+  DataType* type = large_utf8().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRING);
+}
+TEST(TestAdapterWriteConverter, typeBinary) {
+  DataType* type = binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeLargeBinary) {
+  DataType* type = large_binary().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinary) {
+  DataType* type = fixed_size_binary(3).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeBinaryZero) {
+  DataType* type = fixed_size_binary(0).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::BINARY);
+}
+TEST(TestAdapterWriteConverter, typeDate32) {
+  DataType* type = date32().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DATE);
+}
+TEST(TestAdapterWriteConverter, typeDate64) {
+  DataType* type = date64().get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampSecond) {
+  DataType* type = timestamp(TimeUnit::type::SECOND).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMilli) {
+  DataType* type = timestamp(TimeUnit::type::MILLI).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampMicro) {
+  DataType* type = timestamp(TimeUnit::type::MICRO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeTimestampNano) {
+  DataType* type = timestamp(TimeUnit::type::NANO).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::TIMESTAMP);
+}
+TEST(TestAdapterWriteConverter, typeDecimal) {
+  DataType* type = decimal(32, 5).get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::DECIMAL);
+  EXPECT_EQ(out->getPrecision(), 32);
+  EXPECT_EQ(out->getScale(), 5);
+}
+TEST(TestAdapterWriteConverter, typeList) {
+  auto sharedPtrArrowType = list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeLargeList) {
+  auto sharedPtrArrowType = large_list(std::make_shared<Field>("a", int32()));
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeList) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 3);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeFixedSizeListZero) {
+  auto sharedPtrArrowType = fixed_size_list(std::make_shared<Field>("a", int32()), 0);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 1);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::LIST);
+  EXPECT_EQ(out->getSubtype(0)->getKind(), liborc::TypeKind::INT);
+}
+TEST(TestAdapterWriteConverter, typeStructTrivial) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  auto sharedPtrArrowType = struct_(xFields);
+  DataType* type = sharedPtrArrowType.get();
+  ORC_UNIQUE_PTR<liborc::Type> out;
+  (void)(adapters::orc::GetORCType(type, &out));
+  EXPECT_EQ(out->getSubtypeCount(), 0);
+  EXPECT_EQ(out->getKind(), liborc::TypeKind::STRUCT);
+}
+TEST(TestAdapterWriteConverter, typeStructSingleton) {
+  std::vector<std::shared_ptr<Field>> xFields;
+  xFields.push_back(std::make_shared<Field>("a", utf8()));

Review comment:
       Thanks! I have done that.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {

Review comment:
       Ah. Yes.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612990896



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,

Review comment:
       @emkornfield @lidavidm Please correct me if I'm wrong. Since I use the fact that real DATE64 and TIMESTAMP (with UNIT not equals NANO) can be cast to TIMESTAMP (using NANO) without getting beyond int64_t (because ORC essentially only supports NANO, see TimestampVectorBatch in https://orc.apache.org/docs/core-cpp.html) I don't think `arrow::random::RandomArrayGenerator.ArrayOf` can be used.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562266579



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"

Review comment:
       Thanks! This has been fixed.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-819293482


   > Sorry still reviewing, will try to do more tomorrow.
   
   That's fine. I have addressed all the comments you gave and pushed.


-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614522411



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+          GetORCType(*arrow_child_type).ValueOrDie();

Review comment:
       Really thanks!




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-753600486


   Now it is ready for review! I haven't spotted any ORC writer bug in the code base itself since 8 days ago so I think it is likely pretty good.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594782962



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,17 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
-                   int64_t offset, int64_t length, ArrayBuilder* builder);
+                   int64_t offset, int64_t length, arrow::ArrayBuilder* builder);
+
+Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,

Review comment:
       None of the symbols here is actually exposed to library users. Do we need to document these functions?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614590764



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const arrow::Decimal128Type&>(type).scale());
+      return liborc::createDecimalType(precision, scale);
+    }
+    case arrow::Type::type::LIST:
+    case arrow::Type::type::FIXED_SIZE_LIST:
+    case arrow::Type::type::LARGE_LIST: {
+      std::shared_ptr<arrow::DataType> arrow_child_type =
+          static_cast<const arrow::BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orc_subtype =
+          GetORCType(*arrow_child_type).ValueOrDie();

Review comment:
       All instances of `ValueOrDie` has been replaced by `ARROW_ASSIGN_OR_RAISE` or `EXPECT_OK_AND_ASSIGN` for regular code and test code respectively throughout my code base.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773335



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Yup! 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612965289



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       Given that ORC only has these three exposed errors https://github.com/apache/orc/blob/master/c%2B%2B/include/orc/Exceptions.hh ParseError is still more reasonable compared to NotImplementedYet or InvalidArgument.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565774069



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614606429



##########
File path: python/pyarrow/_orc.pyx
##########
@@ -109,3 +113,28 @@ cdef class ORCReader(_Weakrefable):
                 check_status(deref(self.reader).Read(indices, &sp_table))
 
         return pyarrow_wrap_table(sp_table)
+
+cdef class ORCWriter(_Weakrefable):
+    cdef:
+        object source
+        unique_ptr[ORCFileWriter] writer
+
+    def open(self, object source):
+        cdef:
+            shared_ptr[COutputStream] rd_handle
+        self.source = source
+        get_writer(source, &rd_handle)
+        with nogil:
+            self.writer = move(GetResultValue[unique_ptr[ORCFileWriter]](
+                ORCFileWriter.Open(rd_handle.get())))

Review comment:
       Thanks! Given that we use the `pImpl` pattern for `ORCFileWriter` (and in fact `ORCFileReader` as well) I will store it in the Python `ORCWriter` object.




-- 
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591531862



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,103 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    constexpr uint64_t batch_size = 1;  // Doesn't matter what it is

Review comment:
       I don't think it doesn't matter. With `batch_size = 1`, you're basically calling `WriteBatch` once per row * col, this is going to add a lot of overhead.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614575581



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
       On `WriteBatch`: Doc added and parameters rotated so that they make more sense.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614620841



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       Well, I removed a test that compares the ORC schema with some JSON one since they don’t actually line up given the new behavior of the ORC Reader on MAP arrays.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565771826



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       Sure! Because it does not and should not affect the result.
   




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565771907



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);

Review comment:
       Of course! :)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r560628957



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       It is exclusively used by FillStructBatch. The cause is that ORC is much stricter than Arrow in terms of consistency. In this case if a struct scalar is null all its children must be set to null or ORC is not going to function properly. This is why I added incomingMask to pass on null status from a struct to its children.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612929655



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +71,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       nit: fix to use kCamelCase conventions and use constexpr.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612965289



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());

Review comment:
       Given that ORC only has these three errors https://github.com/apache/orc/blob/master/c%2B%2B/include/orc/Exceptions.hh ParseError is still more reasonable compared to NotImplementedYet or InvalidArgument.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612954997



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    std::vector<int64_t> arrow_index_offset(num_cols_, 0);
+    std::vector<int> arrow_chunk_offset(num_cols_, 0);
+    std::unique_ptr<liborc::ColumnVectorBatch> batch =
+        writer_->createRowBatch(kOrcWriterBatchSize);

Review comment:
       Remind me again, is there reason kOrcWriterBatchSize isn't a configurable parameter?




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591678190



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,103 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    constexpr uint64_t batch_size = 1;  // Doesn't matter what it is

Review comment:
       @pitrou Ah..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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567576875



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +197,1930 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// Trivial
+TEST(TestAdapterWriteTrivial, writeZeroRowsNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {R"([])"});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessNoConversion) {
+  std::shared_ptr<Table> table = TableFromJSON(
+      schema({field("bool", boolean()), field("int8", int8()), field("int16", int16()),
+              field("int32", int32()), field("int64", int64()), field("float", float32()),
+              field("double", float64()), field("decimal128nz", decimal(25, 6)),
+              field("decimal128z", decimal(32, 0)), field("date32", date32()),
+              field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+              field("binary", binary()),
+              field("struct", struct_({field("a", utf8()), field("b", int64())})),
+              field("list", list(int32())),
+              field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+      {});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeZeroRowsWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {R"([])"}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {R"([])"});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessWithConversion) {
+  std::shared_ptr<Table>
+      input_table = TableFromJSON(
+          schema({field("date64", date64()), field("ts0", timestamp(TimeUnit::SECOND)),
+                  field("ts1", timestamp(TimeUnit::MILLI)),
+                  field("ts2", timestamp(TimeUnit::MICRO)),
+                  field("large_string", large_utf8()),
+                  field("large_binary", large_binary()),
+                  field("fixed_size_binary0", fixed_size_binary(0)),
+                  field("fixed_size_binary", fixed_size_binary(5)),
+                  field("large_list", large_list(int32())),
+                  field("fixed_size_list", fixed_size_list(int32(), 3)),
+                  field("map", map(utf8(), utf8()))}),
+          {}),
+      expected_output_table = TableFromJSON(
+          schema({field("date64", timestamp(TimeUnit::NANO)),
+                  field("ts0", timestamp(TimeUnit::NANO)),
+                  field("ts1", timestamp(TimeUnit::NANO)),
+                  field("ts2", timestamp(TimeUnit::NANO)), field("large_string", utf8()),
+                  field("large_binary", binary()), field("fixed_size_binary0", binary()),
+                  field("fixed_size_binary", binary()),
+                  field("large_list", list(int32())),
+                  field("fixed_size_list", list(int32())),
+                  field("map",
+                        list(struct_({field("key", utf8()), field("value", utf8())})))}),
+          {});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+                            kDefaultSmallMemStreamSize / 16);
+}
+
+// General
+TEST(TestAdapterWriteGeneral, writeAllNullsNew) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = schema(table_fields);
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  for (int i = 0; i < numCols; i++) {
+    for (int j = 0; j < 5; j++) {
+      int row_count = j % 2 ? 0 : num_rows / 2;
+      arrays[i][j] = rand.ArrayOf(table_fields[i]->type(), row_count, 1);
+    }
+  }
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(table_schema, cv);
+  AssertTableWriteReadEqual(table, table);
+}
+
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> table_fields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("decimal128nz", decimal(33, 4)),
+      field("decimal128z", decimal(35, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary())};
+  std::shared_ptr<Schema> table_schema = std::make_shared<Schema>(table_fields);
+
+  int64_t num_rows = 10000;
+  int64_t numCols = table_fields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));

Review comment:
       it would be good to test decimal256 if possible 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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-789737670


   @mathyingzhou What is the status of this PR? Are you waiting for another review? (feel free to ping=


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561520934



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;

Review comment:
       Thanks! I will fix all of them.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r574939922



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondNanos, 1);
+      }
+    }
+    case arrow::Type::type::DECIMAL:

Review comment:
       Thanks! Fixed!

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561405645



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +225,2478 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 0;
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+
+  MemoryOutputStream mem_stream(DEFAULT_SMALL_MEM_STREAM_SIZE);
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(static_cast<liborc::OutputStream*>(
+          new MemoryOutputStream(DEFAULT_SMALL_MEM_STREAM_SIZE / 16)));
+  ARROW_EXPECT_OK(writer->Open(sharedPtrSchema, out_stream));
+  ARROW_EXPECT_OK(writer->Write(table));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));
+
+  std::unique_ptr<adapters::orc::ORCFileReader> reader;
+  ASSERT_TRUE(
+      adapters::orc::ORCFileReader::Open(in_stream, default_memory_pool(), &reader).ok());
+  std::shared_ptr<Table> outputTable;
+  ARROW_EXPECT_OK(reader->Read(&outputTable));
+  EXPECT_EQ(outputTable->num_columns(), numCols);
+  EXPECT_EQ(outputTable->num_rows(), numRows);
+  EXPECT_TRUE(outputTable->Equals(*table));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);

Review comment:
       > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Thanks! There isn’t anything Parquet-related either. Can that be removed 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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790297527


   @pitrou  Yes now it is ready for another review. I have fixed all the issues you mentioned and shortened the tests to about 650 lines (with more tests!) It should be a lot neater this time.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614558933



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));

Review comment:
       The main reasons why the Slice is here now despite not being present earlier are:
   1. The need to skip elements of `list_array.values()` corresponding to Null lists since ORC does not support or accommodate null List entries with nonzero amount of elements. This is why unlike in struct I called WriteBatch for every nonnull List entry to begin with. 
   2. The need to know the exact array to process using `ArrayDataVisitor`. In order not to have repeated length checks in `VisitNull` and `VisitValue` of `Appender` et al and in order for `Write*Batch` to look neater (it used to have 5-7 arguments) length checks are handled in methods that call WriteBatch as opposed to individual `Write*Batch` or `Visit*` methods.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));

Review comment:
       The main reasons why the Slice is here now despite not being present earlier are:
   1. The need to skip elements of `list_array.values()` corresponding to Null lists since ORC does not support or accommodate null List entries with nonzero amount of elements. This is why unlike in struct I called WriteBatch for every nonnull List entry to begin with. 
   2. The need to know the exact array to process using `ArrayDataVisitor`. In order not to have repeated length checks in `VisitNull` and `VisitValue` of `Appender` et al and in order for `Write*Batch` to look neater (it used to have 5-7 arguments) length checks are handled in methods that call `WriteBatch` as opposed to individual `Write*Batch` or `Visit*` methods.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594465495



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       Fixed.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567579503



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +474,107 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream& output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema;
+    RETURN_NOT_OK(GetORCType(*(table.schema()), &orc_schema));
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    int64_t num_rows = table.num_rows();
+    const int num_cols_ = table.num_columns();
+    const int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       make this static constexpr.  Why doesn't it matter?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595076



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];

Review comment:
       Yes, they are. The batches are reused.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614171868



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,44 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/array.h"
+#include "arrow/buffer.h"
+#include "arrow/buffer_builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/compute/cast.h"

Review comment:
       Fair enough!




-- 
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-817451185


   @pitrou I completely revamped adapter_util.cc and switched to arrow::ArrayDataVisitor. That is, all comments have been addressed. Please review.
   
   By the way I have nothing to do with that Flight error so it shouldn’t prevent a merge.


-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614515778



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                    array.type()->ToString());
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Result<ORC_UNIQUE_PTR<liborc::Type>> GetORCType(const arrow::DataType& type) {
+  arrow::Type::type kind = type.id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+    case arrow::Type::type::INT8:
+      return liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+    case arrow::Type::type::INT16:
+      return liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+    case arrow::Type::type::INT32:
+      return liborc::createPrimitiveType(liborc::TypeKind::INT);
+    case arrow::Type::type::INT64:
+      return liborc::createPrimitiveType(liborc::TypeKind::LONG);
+    case arrow::Type::type::FLOAT:
+      return liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+    case arrow::Type::type::DOUBLE:
+      return liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+      return liborc::createPrimitiveType(liborc::TypeKind::STRING);
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+    case arrow::Type::type::DATE32:
+      return liborc::createPrimitiveType(liborc::TypeKind::DATE);
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+      return liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+    case arrow::Type::type::DECIMAL128: {
+      const uint64_t precision = static_cast<uint64_t>(
+          static_cast<const arrow::Decimal128Type&>(type).precision());

Review comment:
       Fixed. The same fixed has been applied to all similar cases.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614413650



##########
File path: cpp/src/arrow/testing/random.h
##########
@@ -310,6 +310,16 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::shared_ptr<Array> FixedSizeBinary(int64_t size, int32_t byte_width,
                                          double null_probability = 0);
 
+  // /// \brief Generate a random StructArray
+  // ///
+  // /// \param[in] children Vector of Arrays containing the data for each child
+  // /// \param[in] size The size of the generated list array
+  // /// \param[in] null_probability the probability of a list value being null
+  // ///
+  // /// \return a generated Array
+  // std::shared_ptr<Array> Struct(const ArrayVector& children, int64_t size,
+  //                               double null_probability);
+

Review comment:
       Yup. 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614405583



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;

Review comment:
       Sure. 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614556421



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;

Review comment:
       Fixed!




-- 
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.

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



[GitHub] [arrow] lidavidm commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
lidavidm commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r613181812



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,182 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+template <typename T, typename U>
+void randintpartition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,

Review comment:
       You could do something like `GenerateArray(field("", timestamp(TimeUnit::SECOND), key_value_metadata({{"min", kSecondMin}, {"max", kSecondMax}})))` but that isn't really much more concise than what you have here at this point, right, since your constraint is that they all have to be castable to nanoseconds without overflowing.




-- 
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-764050107


   > I didn't see anything rust related in this PR so I removed the Rust label
   
   @alamb Thanks! There is nothing parquet-related in this PR either. Can the parquet label be removed 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561521360



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r563395657



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,

Review comment:
       Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567593744



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondNanos, 1);
+      }
+    }
+    case arrow::Type::type::DECIMAL:
+      return WriteDecimalBatch(column_vector_batch, arrow_offset, orc_offset, length,
+                               array, incoming_mask);
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(column_vector_batch, arrow_offset, orc_offset, length,
+                              array, incoming_mask);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(column_vector_batch, arrow_offset,
+                                              orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(column_vector_batch, arrow_offset, orc_offset, length, array,
+                           incoming_mask);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);

Review comment:
       Using the array's type->ToString() would give a better error.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567578886



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table));
+}
+TEST(TestAdapterWriteGeneral, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(36, 2)),
+                                              field("decimal128z", decimal(31, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(36, 2)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(31, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin[2], string_[13];
+  std::string str;
+  for (int64_t i = 0; i < numRows / 2; i++) {
+    bin[0] = i % 128;
+    bin[1] = bin[0];
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(i % 128));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                        ->Append(Decimal128(std::to_string(i) + ".56")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                        ->Append(Decimal128(std::to_string(i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                        ->Append(INT64_C(1605547718999999999) + i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin, 2));
+  }
+  for (int64_t i = numRows / 2; i < numRows; i++) {
+    bin[0] = i % 256;
+    bin[1] = (i / 256) % 256;
+    str = "Arrow " + std::to_string(3 - 4 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(-(i % 128)));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(4 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                        ->Append(Decimal128(std::to_string(-i) + ".00")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                        ->Append(Decimal128(std::to_string(-i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                        ->Append(INT64_C(1605557718999999999) - i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin, 2));
+  }

Review comment:
       And using the ArrayFromJSON is much clearer for most deterministic inputs.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562260453



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;

Review comment:
       It is. If the root->numElements is not set it is 0 and nothing gets written.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594787933



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,35 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/api.h"

Review comment:
       Ah I really didn't see this one..Fixed.:)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562266451



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/table_builder.h"

Review comment:
       Thanks! This has been fixed.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-793383602


   @pitrou Now all tests have passed. Please review.
   @kou That’s fine haha. Next time I will check these files 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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-751486530


   I have revamped the tests completely and refactored the code to eliminate dependency issues and get all checks to pass. Right now I'm integrating my old nested type tests into adapter_test.cpp. Then I will make this PR ready for review again.
   
   Note that support for dense union and sparse union has been delayed till a further PR since there is no read union in the ORC reader which makes testing hard. In that PR I will probably add the following features:
   
   1. Read and write support for union types.
   2. Replicate orc::WriterOptions in adapters::orc::ORCWriterOptions.
   3. Replicate orc::ReaderOptions in adapters::orc::ORCReaderOptions and add them to the the ORC reader.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r561967537



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +146,35 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] schema of the Arrow table
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \param[out] writer the returned writer object
+  /// \return Status
+  static Status Open(const std::shared_ptr<Schema>& schema,

Review comment:
       Nowadays we would rather return `Result<std::unique_ptr<OrcFileWriter>>` and forego the out-pointer parameter.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");

Review comment:
       Is it ok if it's not an actual filename? Is it just used for error messages and such?

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {

Review comment:
       Generally, everything that's not exposed in a `.h` file should be put in the anonymous namespace, to minimize the size of exported symbols (and potentially open up some optimization opportunities for the compiler).
   
   Since we're doing this, can you also do the same change for the read side?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray);

Review comment:
       Same comments here.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);

Review comment:
       Can use `GetView` as well.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {
+      return st;
+    }
+    if (arrowChunkOffset < numBatch && orcOffset < length) {

Review comment:
       Why repeat this condition? It's the same as in the `while` above.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;

Review comment:
       This is making a copy of the vector. Is it necessary? Or can you just take a const-ref?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {

Review comment:
       Also, please use `checked_pointer_cast`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+ private:
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;

Review comment:
       Our naming scheme mandates `out_stream_` and `orc_schema_` (no camelcase for variable names, including class and struct members).

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);

Review comment:
       Please, no camelCase for variable names :-)

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(

Review comment:
       The method here is returning a `Status`, it shouldn't throw on error.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();

Review comment:
       `num_rows`

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {

Review comment:
       Use a const-ref here.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));

Review comment:
       Since this is returning a Status, you should use `RETURN_NOT_OK` here.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       This should go into the namespaces below and/or be made static (or anonymous).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Can you put all private stuff in the anonymous namespace?

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();

Review comment:
       I find it a bit weird that `WriteTable` implicitly closes the writer. I would expect a separate `Status Close()` method for that. What do you think?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];

Review comment:
       Isn't the batch supposed to have been cleared before calling this function?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>

Review comment:
       Preferably use CamelCase for template parameters (or, at worse, UPPERCASE)...

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));

Review comment:
       You're passing `batch_size` even though `num_rows` may be smaller.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);

Review comment:
       This is making a spurious copy. Please use `GetView` instead.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;

Review comment:
       I'm curious: is this necessary? Does the ORC API need it?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null

Review comment:
       Does ORC really need the terminating null here?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,

Review comment:
       * `arrow_offset` and `orc_offset`. Also, mutable references are generally prohibited. Instead, pass a pointer (which clearly indicates it's gonna be mutated).
   * Why `Array*`? You're not supposed to mutate the array... do you mean `const Array&`?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);

Review comment:
       What is `incomingMask`? Doesn't seem used, is it?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;

Review comment:
       According to the example in `https://orc.apache.org/docs/core-cpp.html`, `numElements` should be the number of rows in _this_ batch only. But `orcOffset` would be the number of rows from the beginning, no?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);

Review comment:
       Can use `GetView`... and then reuse the same (template) function as for `Binary` and `String`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"

Review comment:
       This inclusion shouldn't be necessary.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {

Review comment:
       All these `FillBatch` functions are repeating the same looping and null selection logic that could be factored out (for example in a template function, a class...). Especially since we may want to speed up this up some day.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;

Review comment:
       Why is this commented out?

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>
 #include <vector>
 
+#include "arrow/io/file.h"
 #include "arrow/io/interfaces.h"
 #include "arrow/memory_pool.h"
 #include "arrow/record_batch.h"
 #include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/table_builder.h"

Review comment:
       These inclusions shouldn't be necessary. You can include `arrow/type_fwd.h` for general declarations.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();

Review comment:
       Is it useful to shortcut here?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;

Review comment:
       You can use `util::optional` to avoid the `shared_ptr` allocation.

##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -19,12 +19,16 @@
 
 #include <cstdint>
 #include <memory>
+#include <sstream>

Review comment:
       This doesn't seem used in this `.h` file.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;

Review comment:
       This is the default and no need to be explicit.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;

Review comment:
       So you're doing this properly here and not in other fill functions?

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));

Review comment:
       It would be more readable to use `arrow::util::Decimal128`, IMHO:
   ```c++
   const Decimal128 dec_value(array->GetValue(arrow_offset));
   batch->values[orcOffset] = liborc::Int128(dec_value.high_bits(), dec_value.low_bits());
   ```
   

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       `const DataType&`. The policy is: pass a reference if it's const, a pointer if it's non-const (and will be mutated by the callee).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));

Review comment:
       Please use `nullptr` not `NULLPTR`, except in `.h` files.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();

Review comment:
       You can use `num_fields()`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);

Review comment:
       Hmm... You could use `VisitTypeInline`, for example, to avoid the annoyance of writing all this. You can find an example here: https://github.com/apache/arrow/blob/master/cpp/src/arrow/array/validate.cc#L47

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {

Review comment:
       Just use `RETURN_NOT_OK`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {

Review comment:
       We would probably use `int` or `int64_t`.

##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is

Review comment:
       Make this `const`? Also, I don't understand the comment. Why doesn't it matter?

##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -163,3 +163,15 @@ def test_orcfile_empty(datadir):
         ]))),
     ])
     assert table.schema == expected_schema
+
+
+def test_orcfile_readwrite():
+    from pyarrow import orc
+    buffer_output_stream = pa.BufferOutputStream()
+    a = pa.array([1, None, 3, None])
+    b = pa.array([None, 'Arrow', None, 'ORC'])
+    table = pa.table({"int64": a, "utf8": b})
+    orc.write_table(buffer_output_stream, table)
+    buffer_reader = pa.BufferReader(buffer_output_stream.getvalue())
+    outputTable = orc.ORCFile(buffer_reader).read()
+    assert table.equals(outputTable)

Review comment:
       `output_table`, as per [PEP 8](https://www.python.org/dev/peps/pep-0008/)

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray);

Review comment:
       By the way, the `AppendBatch` vs. `FillBatch` naming is confusing. Can we make the difference explicit? One is writing to ORC, the other is reading from ORC.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +950,137 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  Type::type kind = type.id();
+  switch (kind) {
+    case Type::type::NA: {  // Makes out NULLPTR
+      out->reset();
+      break;
+    }
+    case Type::type::BOOL:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+      break;
+    case Type::type::INT8:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+      break;
+    case Type::type::INT16:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+      break;
+    case Type::type::INT32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::INT);
+      break;
+    case Type::type::INT64:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::LONG);
+      break;
+    case Type::type::FLOAT:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+      break;
+    case Type::type::DOUBLE:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+      break;
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case Type::type::STRING:
+    case Type::type::LARGE_STRING:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::STRING);
+      break;
+    case Type::type::BINARY:
+    case Type::type::LARGE_BINARY:
+    case Type::type::FIXED_SIZE_BINARY:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+      break;
+    case Type::type::DATE32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DATE);
+      break;
+    case Type::type::DATE64:
+    case Type::type::TIMESTAMP:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+      break;
+    case Type::type::DECIMAL128: {
+      const uint64_t precision =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).scale());
+      *out = liborc::createDecimalType(precision, scale);
+      break;
+    }
+    case Type::type::LIST:
+    case Type::type::FIXED_SIZE_LIST:
+    case Type::type::LARGE_LIST: {
+      std::shared_ptr<DataType> arrowChildType =
+          static_cast<const BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+      RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+      *out = liborc::createListType(std::move(orcSubtype));
+      break;
+    }
+    case Type::type::STRUCT: {
+      *out = liborc::createStructType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {
+        std::string fieldName = (*it)->name();
+        std::shared_ptr<DataType> arrowChildType = (*it)->type();
+        ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+        RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+        (*out)->addStructField(fieldName, std::move(orcSubtype));
+      }
+      break;
+    }
+    case Type::type::MAP: {
+      std::shared_ptr<DataType> keyArrowType =
+          checked_cast<const MapType&>(type).key_type();
+      std::shared_ptr<DataType> itemArrowType =
+          checked_cast<const MapType&>(type).item_type();
+      ORC_UNIQUE_PTR<liborc::Type> keyORCType, itemORCType;
+      RETURN_NOT_OK(GetORCType(*keyArrowType, &keyORCType));
+      RETURN_NOT_OK(GetORCType(*itemArrowType, &itemORCType));
+      *out = liborc::createMapType(std::move(keyORCType), std::move(itemORCType));
+      break;
+    }
+    case Type::type::DENSE_UNION:
+    case Type::type::SPARSE_UNION: {
+      *out = liborc::createUnionType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const UnionType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {

Review comment:
       Same as above...

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,
+                         const int maxSize = DEFAULT_SMALL_MEM_STREAM_SIZE) {
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(
+          static_cast<liborc::OutputStream*>(new MemoryOutputStream(maxSize)));
+  ARROW_EXPECT_OK(writer->Open(tableIn->schema(), out_stream));
+  ARROW_EXPECT_OK(writer->Write(tableIn));
+  auto output_mem_stream = static_cast<MemoryOutputStream*>(writer->ReleaseOutStream());
+  std::shared_ptr<io::RandomAccessFile> in_stream(
+      new io::BufferReader(std::make_shared<Buffer>(
+          reinterpret_cast<const uint8_t*>(output_mem_stream->getData()),
+          static_cast<int64_t>(output_mem_stream->getLength()))));

Review comment:
       For concision, you can write: `auto in_stream = std::make_shared<io::BufferReader>(...)`

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,
+                         const int maxSize = DEFAULT_SMALL_MEM_STREAM_SIZE) {
+  std::unique_ptr<ORCMemWriter> writer =
+      std::unique_ptr<ORCMemWriter>(new ORCMemWriter());
+  std::unique_ptr<liborc::OutputStream> out_stream =
+      std::unique_ptr<liborc::OutputStream>(
+          static_cast<liborc::OutputStream*>(new MemoryOutputStream(maxSize)));

Review comment:
       What is `MemoryOutputStream`? You should use Arrow's `BufferOutputStream`.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -425,6 +950,137 @@ Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
   return Status::OK();
 }
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out) {
+  Type::type kind = type.id();
+  switch (kind) {
+    case Type::type::NA: {  // Makes out NULLPTR
+      out->reset();
+      break;
+    }
+    case Type::type::BOOL:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BOOLEAN);
+      break;
+    case Type::type::INT8:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BYTE);
+      break;
+    case Type::type::INT16:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::SHORT);
+      break;
+    case Type::type::INT32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::INT);
+      break;
+    case Type::type::INT64:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::LONG);
+      break;
+    case Type::type::FLOAT:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::FLOAT);
+      break;
+    case Type::type::DOUBLE:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DOUBLE);
+      break;
+    // Use STRING instead of VARCHAR for now, both use UTF-8
+    case Type::type::STRING:
+    case Type::type::LARGE_STRING:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::STRING);
+      break;
+    case Type::type::BINARY:
+    case Type::type::LARGE_BINARY:
+    case Type::type::FIXED_SIZE_BINARY:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::BINARY);
+      break;
+    case Type::type::DATE32:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::DATE);
+      break;
+    case Type::type::DATE64:
+    case Type::type::TIMESTAMP:
+      *out = liborc::createPrimitiveType(liborc::TypeKind::TIMESTAMP);
+      break;
+    case Type::type::DECIMAL128: {
+      const uint64_t precision =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).precision());
+      const uint64_t scale =
+          static_cast<uint64_t>(static_cast<const Decimal128Type&>(type).scale());
+      *out = liborc::createDecimalType(precision, scale);
+      break;
+    }
+    case Type::type::LIST:
+    case Type::type::FIXED_SIZE_LIST:
+    case Type::type::LARGE_LIST: {
+      std::shared_ptr<DataType> arrowChildType =
+          static_cast<const BaseListType&>(type).value_type();
+      ORC_UNIQUE_PTR<liborc::Type> orcSubtype;
+      RETURN_NOT_OK(GetORCType(*arrowChildType, &orcSubtype));
+      *out = liborc::createListType(std::move(orcSubtype));
+      break;
+    }
+    case Type::type::STRUCT: {
+      *out = liborc::createStructType();
+      std::vector<std::shared_ptr<Field>> arrowFields =
+          checked_cast<const StructType&>(type).fields();
+      for (std::vector<std::shared_ptr<Field>>::iterator it = arrowFields.begin();
+           it != arrowFields.end(); ++it) {

Review comment:
       Please rephrase this as:
   ```c++
   for (const auto& field : checked_cast<const StructType&>(type).fields()) {
     ...
   ```

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);

Review comment:
       Similarly, return `Result<ORC_UNIQUE_PTR<liborc::Type>>`

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(

Review comment:
       This is already in `ORCWriter`, why are you re-writing this?

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +73,81 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+class ORCMemWriter {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              ORC_UNIQUE_PTR<liborc::OutputStream>& outStream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = std::move(outStream);
+    ARROW_EXPECT_OK(adapters::orc::GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ARROW_EXPECT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));
+      }
+      root->numElements = fields[0]->numElements;
+      writer_->add(*batch);
+      batch->clear();
+      numRows -= batch_size;
+    }
+    writer_->close();
+    return Status::OK();
+  }
+
+  liborc::OutputStream* ReleaseOutStream() { return outStream_.release(); }
+
+  ORC_UNIQUE_PTR<liborc::Writer> writer_;
+  std::unique_ptr<liborc::WriterOptions> orc_options_;
+  std::shared_ptr<Schema> schema_;
+  ORC_UNIQUE_PTR<liborc::OutputStream> outStream_;
+  ORC_UNIQUE_PTR<liborc::Type> orcSchema_;
+  int num_cols_;
+};
+
+bool tableWriteReadEqual(const std::shared_ptr<Table>& tableIn,
+                         const std::shared_ptr<Table>& predictedTableOut,

Review comment:
       Our convention is "expected", not "predicted" :-)

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,35 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/api.h"

Review comment:
       `api.h` files are much too heavyweight, please make the inclusions more fine-grained.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeChunkless) {
+  std::vector<std::shared_ptr<Field>> xFieldsSub{std::make_shared<Field>("a", utf8()),
+                                                 std::make_shared<Field>("b", int32())};
+  std::vector<std::shared_ptr<Field>> xFields{
+      field("bool", boolean()),
+      field("int8", int8()),
+      field("int16", int16()),
+      field("int32", int32()),
+      field("int64", int64()),
+      field("float", float32()),
+      field("double", float64()),
+      field("decimal128nz", decimal(25, 6)),
+      field("decimal128z", decimal(32, 0)),
+      field("date32", date32()),
+      field("ts3", timestamp(TimeUnit::NANO)),
+      field("string", utf8()),
+      field("binary", binary()),
+      field("struct", struct_(xFieldsSub)),
+      field("list", list(int32())),
+      field("lsl", list(struct_({field("lsl0", list(int32()))})))};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  ArrayMatrix av(numCols, ArrayVector(0, NULLPTR));
+
+  for (int col = 0; col < numCols; col++) {
+    cv.push_back(std::make_shared<ChunkedArray>(av[col], xFields[col]->type()));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table, DEFAULT_SMALL_MEM_STREAM_SIZE / 16));
+}
+TEST(TestAdapterWriteGeneral, writeAllNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(33, 4)),
+                                              field("decimal128z", decimal(35, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(33, 4)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(35, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  for (int i = 0; i < numRows; i++) {
+    int chunk = i < (numRows / 2) ? 1 : 3;
+    for (int col = 0; col < numCols; col++) {
+      ARROW_EXPECT_OK(builders[col][chunk]->AppendNull());
+    }
+  }
+
+  ArrayMatrix arrays(numCols, ArrayVector(5, NULLPTR));
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    for (int i = 0; i < 5; i++) {
+      ARROW_EXPECT_OK(builders[col][i]->Finish(&arrays[col][i]));
+    }
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));
+  }
+
+  std::shared_ptr<Table> table = Table::Make(sharedPtrSchema, cv);
+  EXPECT_TRUE(tableWriteReadEqual(table, table));
+}
+TEST(TestAdapterWriteGeneral, writeNoNulls) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("decimal128nz", decimal(36, 2)),
+                                              field("decimal128z", decimal(31, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numRows = 10000;
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderMatrix builders(numCols, ArrayBuilderVector(5, NULLPTR));
+
+  for (int i = 0; i < 5; i++) {
+    builders[0][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+    builders[1][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+    builders[2][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+    builders[3][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+    builders[4][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+    builders[5][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(36, 2)));
+    builders[6][i] = std::static_pointer_cast<ArrayBuilder>(
+        std::make_shared<Decimal128Builder>(decimal(31, 0)));
+    builders[7][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+    builders[8][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+            timestamp(TimeUnit::NANO), default_memory_pool()));
+    builders[9][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+    builders[10][i] =
+        std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  }
+
+  char bin[2], string_[13];
+  std::string str;
+  for (int64_t i = 0; i < numRows / 2; i++) {
+    bin[0] = i % 128;
+    bin[1] = bin[0];
+    str = "Arrow " + std::to_string(2 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][1])->Append(true));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][1])->Append(i % 128));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int16Builder>(builders[2][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][1])->Append(i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][1])
+                        ->Append(Decimal128(std::to_string(i) + ".56")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][1])
+                        ->Append(Decimal128(std::to_string(i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][1])->Append(18600 + i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][1])
+                        ->Append(INT64_C(1605547718999999999) + i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][1])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][1])->Append(bin, 2));
+  }
+  for (int64_t i = numRows / 2; i < numRows; i++) {
+    bin[0] = i % 256;
+    bin[1] = (i / 256) % 256;
+    str = "Arrow " + std::to_string(3 - 4 * i);
+    snprintf(string_, sizeof(string_), "%s", str.c_str());
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BooleanBuilder>(builders[0][3])->Append(false));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int8Builder>(builders[1][3])->Append(-(i % 128)));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Int16Builder>(builders[2][3])->Append(4 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int32Builder>(builders[3][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Int64Builder>(builders[4][3])->Append(-i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[5][3])
+                        ->Append(Decimal128(std::to_string(-i) + ".00")));
+    ARROW_EXPECT_OK(std::static_pointer_cast<Decimal128Builder>(builders[6][3])
+                        ->Append(Decimal128(std::to_string(-i))));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<Date32Builder>(builders[7][3])->Append(18600 - i));
+    ARROW_EXPECT_OK(std::static_pointer_cast<TimestampBuilder>(builders[8][3])
+                        ->Append(INT64_C(1605557718999999999) - i));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<StringBuilder>(builders[9][3])->Append(string_));
+    ARROW_EXPECT_OK(
+        std::static_pointer_cast<BinaryBuilder>(builders[10][3])->Append(bin, 2));
+  }

Review comment:
       I hope you can find a way to cut down on all this test setup. For example, `arrow/testing/random.h` provides utilities to generate random array data.

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);

Review comment:
       `auto schema = ::arrow::schema(fields);`

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       There doesn't seem to be much of a difference, though. The main one seems to be that the null terminator is appended for string data (why is that?).

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  liborc::ColumnVectorBatch* keyBatch = (batch->keys).get();
+  liborc::ColumnVectorBatch* elementBatch = (batch->elements).get();
+  Array* keyArray = array->keys().get();
+  Array* elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (orcOffset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {
+  Type::type kind = type->id();
+  switch (kind) {
+    case Type::type::BOOL:
+      return FillNumericBatchCast<BooleanArray, liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::INT8:
+      return FillNumericBatchCast<NumericArray<arrow::Int8Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT16:
+      return FillNumericBatchCast<NumericArray<arrow::Int16Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT32:
+      return FillNumericBatchCast<NumericArray<arrow::Int32Type>, liborc::LongVectorBatch,
+                                  int64_t>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::INT64:
+      return FillNumericBatch<NumericArray<arrow::Int64Type>, liborc::LongVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::FLOAT:
+      return FillNumericBatchCast<NumericArray<arrow::FloatType>,
+                                  liborc::DoubleVectorBatch, double>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DOUBLE:
+      return FillNumericBatch<NumericArray<arrow::DoubleType>, liborc::DoubleVectorBatch>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::BINARY:
+      return FillBinaryBatch<BinaryArray, int32_t>(type, cbatch, arrowOffset, orcOffset,
+                                                   length, parray, incomingMask);
+    case Type::type::LARGE_BINARY:
+      return FillBinaryBatch<LargeBinaryArray, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::STRING:
+      return FillStringBatch<StringArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                          parray, incomingMask);
+    case Type::type::LARGE_STRING:
+      return FillStringBatch<LargeStringArray>(type, cbatch, arrowOffset, orcOffset,
+                                               length, parray, incomingMask);
+    case Type::type::FIXED_SIZE_BINARY:
+      return FillFixedSizeBinaryBatch(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::DATE32:
+      return FillNumericBatchCast<NumericArray<arrow::Date32Type>,
+                                  liborc::LongVectorBatch, int64_t>(
+          type, cbatch, arrowOffset, orcOffset, length, parray, incomingMask);
+    case Type::type::DATE64:
+      return FillDate64Batch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::TIMESTAMP:
+      return FillTimestampBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                incomingMask);
+    case Type::type::DECIMAL:
+      return FillDecimalBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                              incomingMask);
+    case Type::type::STRUCT:
+      return FillStructBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                             incomingMask);
+    case Type::type::LIST:
+      return FillListBatch<ListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                      parray, incomingMask);
+    case Type::type::LARGE_LIST:
+      return FillListBatch<LargeListArray>(type, cbatch, arrowOffset, orcOffset, length,
+                                           parray, incomingMask);
+    case Type::type::FIXED_SIZE_LIST:
+      return FillFixedSizeListBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                                    incomingMask);
+    case Type::type::MAP:
+      return FillMapBatch(type, cbatch, arrowOffset, orcOffset, length, parray,
+                          incomingMask);
+    default: {
+      return Status::Invalid("Unknown or unsupported Arrow type kind: ", kind);
+    }
+  }
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t length,
+                 ChunkedArray* pchunkedArray) {
+  int numBatch = pchunkedArray->num_chunks();
+  int64_t orcOffset = 0;
+  Status st;
+  while (arrowChunkOffset < numBatch && orcOffset < length) {
+    st = FillBatch(type, cbatch, arrowIndexOffset, orcOffset, length,
+                   pchunkedArray->chunk(arrowChunkOffset).get(), NULLPTR);
+    if (!st.ok()) {
+      return st;
+    }
+    if (arrowChunkOffset < numBatch && orcOffset < length) {
+      arrowIndexOffset = 0;
+      arrowChunkOffset++;
+    }
+  }
+  return Status::OK();
+}
+
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out) {
-  // When subselecting fields on read, liborc will set some nodes to nullptr,
-  // so we need to check for nullptr before progressing
-  if (type == nullptr) {
+  // When subselecting fields on read, liborc will set some nodes to NULLPTR,
+  // so we need to check for NULLPTR before progressing
+  if (type == NULLPTR) {

Review comment:
       Can you undo the `NULLPTR` changes?

##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));

Review comment:
       You can probably use `ArrayFromJSON` and `ChunkedArrayFromJSON` to make such test setup much terser.

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
 constexpr int64_t kOneSecondNanos = 1000000000LL;
+// Jan 1st 2015 in UNIX timestamp
+// constexpr int64_t kConverter = 1420070400LL;
 
 Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
                          int64_t offset, int64_t length, ArrayBuilder* abuilder) {
   auto builder = checked_cast<StructBuilder*>(abuilder);
   auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
       It's not about fixing it, it's about undoing unnecessary changes in this PR ;-)

##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+    return Status::OK();
+  }
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;

Review comment:
       According to the example in https://orc.apache.org/docs/core-cpp.html, `numElements` should be the number of rows in _this_ batch only. But `orcOffset` would be the number of rows from the beginning, no?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-793383602


   @pitrou Now all tests have passed. Please review. Thanks!
   @kou That’s fine haha. Next time I will check these files 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772513



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,

Review comment:
       Thanks! Fixed!




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614550532



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(

Review comment:
       Hmm this breaks the Struct of Struct test. There must be some subtle difference between the two.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614414590



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
       Hmm what exactly is the issue with ArrowOutputStream? 




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614743037



##########
File path: python/pyarrow/tests/test_orc.py
##########
@@ -26,140 +24,13 @@
 pytestmark = pytest.mark.orc
 
 
-try:

Review comment:
       @pitrou Fair enough. Well, back then in Dec and Jan I did manual tests using the ORC adapter to write ORC files and then loaded them using pyorc and compared the results. We should be good.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512702



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {
+      return array;
+    }
+    case arrow::Type::type::STRUCT: {
+      if (array->null_count() == 0) {
+        return array;
+      } else {
+        auto struct_array = std::static_pointer_cast<arrow::StructArray>(array);
+        const std::shared_ptr<arrow::Buffer> bitmap = struct_array->null_bitmap();
+        std::shared_ptr<arrow::DataType> struct_type = struct_array->type();
+        std::size_t size = struct_type->fields().size();
+        std::vector<std::shared_ptr<arrow::Array>> new_children(size, nullptr);
+        for (std::size_t i = 0; i < size; i++) {
+          std::shared_ptr<arrow::Array> child = struct_array->field(i);
+          const std::shared_ptr<arrow::Buffer> child_bitmap = child->null_bitmap();
+          std::shared_ptr<arrow::Buffer> final_child_bitmap;
+          if (child_bitmap == nullptr) {
+            final_child_bitmap = bitmap;
+          } else {
+            final_child_bitmap = arrow::internal::BitmapAnd(
+                                     arrow::default_memory_pool(), bitmap->data(), 0,
+                                     child_bitmap->data(), 0, struct_array->length(), 0)
+                                     .ValueOrDie();
+          }
+          std::shared_ptr<arrow::ArrayData> child_array_data = child->data();
+          std::vector<std::shared_ptr<arrow::Buffer>> child_buffers =
+              child_array_data->buffers;
+          child_buffers[0] = final_child_bitmap;
+          std::shared_ptr<arrow::ArrayData> new_child_array_data = arrow::ArrayData::Make(
+              child->type(), child->length(), child_buffers, child_array_data->child_data,
+              child_array_data->dictionary);
+          new_children[i] = NormalizeArray(arrow::MakeArray(new_child_array_data));
+        }
+        return std::make_shared<arrow::StructArray>(struct_type, struct_array->length(),
+                                                    new_children, bitmap);
+      }
+    }
+    case arrow::Type::type::LIST: {
+      auto list_array = std::static_pointer_cast<arrow::ListArray>(array);
+      return std::make_shared<arrow::ListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::LARGE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::LargeListArray>(array);
+      return std::make_shared<arrow::LargeListArray>(
+          list_array->type(), list_array->length(), list_array->value_offsets(),
+          NormalizeArray(list_array->values()), list_array->null_bitmap());
+    }
+    case arrow::Type::type::FIXED_SIZE_LIST: {
+      auto list_array = std::static_pointer_cast<arrow::FixedSizeListArray>(array);
+      return std::make_shared<arrow::FixedSizeListArray>(
+          list_array->type(), list_array->length(), NormalizeArray(list_array->values()),
+          list_array->null_bitmap());
+    }
+    case arrow::Type::type::MAP: {
+      auto map_array = std::static_pointer_cast<arrow::MapArray>(array);
+      return std::make_shared<arrow::MapArray>(
+          map_array->type(), map_array->length(), map_array->value_offsets(),
+          NormalizeArray(map_array->keys()), NormalizeArray(map_array->items()),
+          map_array->null_bitmap());
+    }
+    default: {
+      return array;
+    }
+  }
+}
+
+template <class DataType, class BatchType, typename Enable = void>
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template <typename T>
+using is_generic_type =
+    std::integral_constant<bool, arrow::is_number_type<T>::value ||
+                                     std::is_same<arrow::Date32Type, T>::value ||
+                                     arrow::is_boolean_type<T>::value>;
+template <typename T, typename R = void>
+using enable_if_generic = arrow::enable_if_t<is_generic_type<T>::value, R>;
+
+// Number-like
+template <class DataType, class BatchType>
+struct Appender<DataType, BatchType, enable_if_generic<DataType>> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using ValueType = typename arrow::TypeTraits<DataType>::CType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(ValueType v) {
+    batch->data[running_orc_offset] = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  BatchType* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Binary
+template <class DataType>
+struct Appender<DataType, liborc::StringVectorBatch> {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  using COffsetType = typename arrow::TypeTraits<DataType>::OffsetType::c_type;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    COffsetType data_length = 0;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset, &data_length)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Decimal
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal64VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    int64_t* lower_bits = reinterpret_cast<int64_t*>(raw_int128);
+    batch->values[running_orc_offset] = *lower_bits;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal64VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+template <>
+struct Appender<arrow::Decimal128Type, liborc::Decimal128VectorBatch> {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    uint8_t* raw_int128 = const_cast<uint8_t*>(array.GetValue(running_arrow_offset));
+    uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+    int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+    batch->values[running_orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::Decimal128Array& array;
+  liborc::Decimal128VectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+};
+
+// Date64 and Timestamp
+template <class DataType>
+struct TimestampAppender {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(int64_t v) {
+    int64_t data = array.Value(running_arrow_offset);
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] =
+        static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+    batch->nanoseconds[running_orc_offset] =
+        (data - conversion_factor_from_second * batch->data[running_orc_offset]) *
+        conversion_factor_to_nano;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const ArrayType& array;
+  liborc::TimestampVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  int64_t conversion_factor_from_second, conversion_factor_to_nano;
+};
+
+// FSB
+struct FixedSizeBinaryAppender {
+  arrow::Status VisitNull() {
+    batch->notNull[running_orc_offset] = false;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  arrow::Status VisitValue(arrow::util::string_view v) {
+    batch->notNull[running_orc_offset] = true;
+    batch->data[running_orc_offset] = reinterpret_cast<char*>(
+        const_cast<uint8_t*>(array.GetValue(running_arrow_offset)));
+    batch->length[running_orc_offset] = data_length;
+    running_orc_offset++;
+    running_arrow_offset++;
+    return arrow::Status::OK();
+  }
+  const arrow::FixedSizeBinaryArray& array;
+  liborc::StringVectorBatch* batch;
+  int64_t running_orc_offset, running_arrow_offset;
+  const int32_t data_length;
+};
+
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class DataType, class BatchType>
+arrow::Status WriteGenericBatch(const arrow::Array& array, int64_t orc_offset,
+                                liborc::ColumnVectorBatch* column_vector_batch) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  Appender<DataType, BatchType> appender{array_, batch, orc_offset, 0};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+template <class DataType>
+arrow::Status WriteTimestampBatch(const arrow::Array& array, int64_t orc_offset,
+                                  liborc::ColumnVectorBatch* column_vector_batch,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  using ArrayType = typename arrow::TypeTraits<DataType>::ArrayType;
+  const ArrayType& array_(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  TimestampAppender<DataType> appender{array_,
+                                       batch,
+                                       orc_offset,
+                                       0,
+                                       conversion_factor_from_second,
+                                       conversion_factor_to_nano};
+  arrow::ArrayDataVisitor<DataType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(const arrow::Array& array, int64_t orc_offset,
+                                        liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::FixedSizeBinaryArray& array_(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  FixedSizeBinaryAppender appender{array_, batch, orc_offset, 0, array_.byte_width()};
+  arrow::ArrayDataVisitor<arrow::FixedSizeBinaryType> visitor;
+  RETURN_NOT_OK(visitor.Visit(*(array_.data()), &appender));
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(const arrow::Array& array, int64_t orc_offset,
+                               liborc::ColumnVectorBatch* column_vector_batch,
+                               bool normalized) {
+  std::shared_ptr<arrow::Array> array_ = arrow::MakeArray(array.data());
+  std::shared_ptr<arrow::StructArray> struct_array(
+      std::static_pointer_cast<arrow::StructArray>(array_));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+    } else {
+      batch->notNull[running_orc_offset] = true;
+    }
+  }
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    batch->fields[i]->resize(orc_offset + arrow_length);
+    RETURN_NOT_OK(
+        WriteBatch(*(struct_array->field(i)), orc_offset, batch->fields[i], true));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(const arrow::Array& array, int64_t orc_offset,
+                             liborc::ColumnVectorBatch* column_vector_batch) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          list_array.value_offset(running_arrow_offset + 1) -
+          list_array.value_offset(running_arrow_offset);
+      element_batch->resize(batch->offsets[running_orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              subarray_orc_length =
+                  batch->offsets[running_orc_offset + 1] - subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(
+          *(list_array.values()->Slice(subarray_arrow_offset, subarray_orc_length)),
+          subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(const arrow::Array& array, int64_t orc_offset,
+                            liborc::ColumnVectorBatch* column_vector_batch) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  int64_t running_arrow_offset = 0, running_orc_offset = orc_offset;
+  if (orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count()) {
+    batch->hasNulls = true;
+  }
+  for (; running_arrow_offset < arrow_length;
+       running_orc_offset++, running_arrow_offset++) {
+    if (array.IsNull(running_arrow_offset)) {
+      batch->notNull[running_orc_offset] = false;
+      batch->offsets[running_orc_offset + 1] = batch->offsets[running_orc_offset];
+    } else {
+      batch->notNull[running_orc_offset] = true;
+      batch->offsets[running_orc_offset + 1] =
+          batch->offsets[running_orc_offset] +
+          map_array.value_offset(running_arrow_offset + 1) -
+          map_array.value_offset(running_arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(running_arrow_offset),
+              subarray_orc_offset = batch->offsets[running_orc_offset],
+              new_subarray_orc_offset = batch->offsets[running_orc_offset + 1],
+              subarray_orc_length = new_subarray_orc_offset - subarray_orc_offset;
+      key_batch->resize(new_subarray_orc_offset);
+      element_batch->resize(new_subarray_orc_offset);
+      RETURN_NOT_OK(
+          WriteBatch(*(key_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, key_batch));
+      RETURN_NOT_OK(
+          WriteBatch(*(element_array->Slice(subarray_arrow_offset, subarray_orc_length)),
+                     subarray_orc_offset, element_batch));
+    }
+  }
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(const arrow::Array& array, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized) {
+  arrow::Type::type kind = array.type_id();
+  column_vector_batch->numElements = orc_offset;
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteGenericBatch<arrow::BooleanType, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT8:
+      return WriteGenericBatch<arrow::Int8Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT16:
+      return WriteGenericBatch<arrow::Int16Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT32:
+      return WriteGenericBatch<arrow::Int32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::INT64:
+      return WriteGenericBatch<arrow::Int64Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FLOAT:
+      return WriteGenericBatch<arrow::FloatType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DOUBLE:
+      return WriteGenericBatch<arrow::DoubleType, liborc::DoubleVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::BINARY:
+      return WriteGenericBatch<arrow::BinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteGenericBatch<arrow::LargeBinaryType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::STRING:
+      return WriteGenericBatch<arrow::StringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteGenericBatch<arrow::LargeStringType, liborc::StringVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE32:
+      return WriteGenericBatch<arrow::Date32Type, liborc::LongVectorBatch>(
+          array, orc_offset, column_vector_batch);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Type>(
+          array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, 1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondMicros, kOneMicroNanos);
+        case arrow::TimeUnit::type::NANO:
+          return WriteTimestampBatch<arrow::TimestampType>(
+              array, orc_offset, column_vector_batch, kOneSecondNanos, 1);
+        default:
+          return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",
+                                        array.type()->ToString());
+      }
+    }
+    case arrow::Type::type::DECIMAL128: {
+      int32_t precision =
+          std::static_pointer_cast<arrow::Decimal128Type>(array.type())->precision();
+      if (precision > 18) {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal128VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      } else {
+        return WriteGenericBatch<arrow::Decimal128Type, liborc::Decimal64VectorBatch>(
+            array, orc_offset, column_vector_batch);
+      }
+    }
+    case arrow::Type::type::STRUCT:
+      return WriteStructBatch(array, orc_offset, column_vector_batch, normalized);
+    case arrow::Type::type::LIST:
+      return WriteListBatch<arrow::ListArray>(array, orc_offset, column_vector_batch);
+    case arrow::Type::type::LARGE_LIST:
+      return WriteListBatch<arrow::LargeListArray>(array, orc_offset,
+                                                   column_vector_batch);
+    case arrow::Type::type::FIXED_SIZE_LIST:
+      return WriteListBatch<arrow::FixedSizeListArray>(array, orc_offset,
+                                                       column_vector_batch);
+    case arrow::Type::type::MAP:
+      return WriteMapBatch(array, orc_offset, column_vector_batch);
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow type: ",

Review comment:
       Thanks! Switched to `Status::NotImplemented` since we don't have dictionary or union support in this PR but they will be supported. For an earlier Timestamp case I switched to `Status::TypeError` since if it reaches that line there is necessarily a type error.




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-796515523


   @pitrou These two errors should be intermittent and have nothing to do with ORC. So please review again. Thanks!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792154086


   @pitrou I have found the cause of the problem which is me failing to account for the possibility of orc::Decimal64VectorBatch being used. I will edit my writer to account for that.
   
   After this PR I'd like to work with you on adapter/testing/random. As you said before tests need to be added. Moreover we should expand random things to RecordBatches, ChunkedArrays & Tables as well as all types. After that we should simplify ORC, Parquet and other 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.

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



[GitHub] [arrow] nevi-me commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
nevi-me commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-759497911


   Hi @mathyingzhou I see that you didn't make changes to the Rust code. Please rebase with `git rebase origin/master` (if `origin` is `apache/arrow`) so that you can remove the Rust changes from the PR. Then you'll need to force-push onto your branch (`git push --force`). Thanks


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-777997864


   Looks like in order toimplement ARROW-11117 I need to make some Ruby changes to the C Glib 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.

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



[GitHub] [arrow] pitrou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591524221



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -77,6 +68,15 @@ namespace orc {
   ORC_ASSIGN_OR_THROW_IMPL(ARROW_ASSIGN_OR_RAISE_NAME(_error_or_value, __COUNTER__), \
                            lhs, rexpr);
 
+const uint64_t ORC_NATURAL_WRITE_SIZE = 128 * 1024;  // Required by liborc::Outstream

Review comment:
       Hmm, @mathyingzhou I don't know if you overlooked this review comment. There are a number of other ones remaining below too.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r578946643



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);

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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772573



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##########
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr<DataType>* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR<liborc::Type>* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR<liborc::Type>* out);
+
 Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
                    int64_t offset, int64_t length, ArrayBuilder* builder);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,

Review comment:
       Thanks! This has been revamped.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-773880147


   I will make some further changes to the code to address Micah Kornfield's concerns but pretty much that's it. Further simplification of the tests require serious expansion of arrow/testing/random and arrow/compute/cast to include nested types (and in the case of the latter at least the fixed_size_binary -> binary cast) or it will remain very ugly. If possible I can work on such features in at least 2 separate PRs.


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614561787



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -58,6 +82,189 @@ class MemoryOutputStream : public liborc::OutputStream {
   uint64_t length_, natural_write_size_;
 };
 
+std::shared_ptr<Buffer> GenerateFixedDifferenceBuffer(int32_t fixed_length,
+                                                      int64_t length) {
+  BufferBuilder builder;
+  int32_t offsets[length];
+  ARROW_EXPECT_OK(builder.Resize(4 * length));
+  for (int32_t i = 0; i < length; i++) {
+    offsets[i] = fixed_length * i;
+  }
+  ARROW_EXPECT_OK(builder.Append(offsets, 4 * length));
+  std::shared_ptr<Buffer> buffer;
+  ARROW_EXPECT_OK(builder.Finish(&buffer));
+  return buffer;
+}
+
+std::shared_ptr<Array> CastFixedSizeBinaryArrayToBinaryArray(
+    std::shared_ptr<Array> array) {
+  auto fixed_size_binary_array = std::static_pointer_cast<FixedSizeBinaryArray>(array);
+  std::shared_ptr<Buffer> value_offsets = GenerateFixedDifferenceBuffer(
+      fixed_size_binary_array->byte_width(), array->length() + 1);
+  return std::make_shared<BinaryArray>(array->length(), value_offsets,
+                                       array->data()->buffers[1],
+                                       array->data()->buffers[0]);
+}
+
+template <typename TargetArrayType>
+std::shared_ptr<Array> CastInt64ArrayToTemporalArray(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array> array) {
+  std::shared_ptr<ArrayData> new_array_data =
+      ArrayData::Make(type, array->length(), array->data()->buffers);
+  return std::make_shared<TargetArrayType>(new_array_data);
+}
+
+/// \brief Construct a random weak composition of a nonnegative integer
+/// i.e. a way of writing it as the sum of a sequence of n non-negative
+/// integers.
+///
+/// \param[in] n the number of integers in the weak composition
+/// \param[in] sum the integer of which a random weak composition is generated
+/// \param[out] out The generated weak composition
+template <typename T, typename U>
+void RandWeakComposition(int64_t n, T sum, std::vector<U>* out) {
+  const int random_seed = 0;
+  std::default_random_engine gen(random_seed);
+  out->resize(n, static_cast<T>(0));
+  T remaining_sum = sum;
+  std::generate(out->begin(), out->end() - 1, [&gen, &remaining_sum] {
+    std::uniform_int_distribution<T> d(static_cast<T>(0), remaining_sum);
+    auto res = d(gen);
+    remaining_sum -= res;
+    return static_cast<U>(res);
+  });
+  (*out)[n - 1] += remaining_sum;
+  std::random_shuffle(out->begin(), out->end());
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomDate64Array(int64_t size,
+                                                         double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  return CastInt64ArrayToTemporalArray<Date64Array>(
+      date64(), rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+}
+
+Result<std::shared_ptr<Array>> GenerateRandomTimestampArray(int64_t size,
+                                                            arrow::TimeUnit::type type,
+                                                            double null_probability) {
+  arrow::random::RandomArrayGenerator rand(kRandomSeed);
+  switch (type) {
+    case arrow::TimeUnit::type::SECOND: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::SECOND),
+          rand.Int64(size, kSecondMin, kSecondMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MILLI: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MILLI),
+          rand.Int64(size, kMilliMin, kMilliMax, null_probability));
+    }
+    case arrow::TimeUnit::type::MICRO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::MICRO),
+          rand.Int64(size, kMicroMin, kMicroMax, null_probability));
+    }
+    case arrow::TimeUnit::type::NANO: {
+      return CastInt64ArrayToTemporalArray<TimestampArray>(
+          timestamp(TimeUnit::NANO),
+          rand.Int64(size, kNanoMin, kNanoMax, null_probability));
+    }
+    default: {
+      return arrow::Status::Invalid("Unknown or unsupported Arrow TimeUnit: ", type);
+    }
+  }
+}
+
+std::shared_ptr<ChunkedArray> GenerateRandomChunkedArray(
+    const std::shared_ptr<DataType>& data_type, int64_t size, int64_t min_num_chunks,
+    int64_t max_num_chunks, double null_probability) {

Review comment:
       Wait. This function itself actually contains ORC-specific code such as the requirement that Date64 and Timestamp scalars must not overflow when cast to Timestamp NANO. Unless this requirement is actually universal in which case we should change how random arrays are canonically generated for these types we shouldn't really leave some function so ORC-specific in `testing/random.h`.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567591673



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it

Review comment:
       Does Decimal256 throw an error anyplace today?




----------------------------------------------------------------
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.

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



[GitHub] [arrow] xhochy commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
xhochy commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562161018



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       Please add this as a comment somewhere in the code, I couldn't easily figure it out by reading the code initially. With this comment it makes sense directly.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567581092



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -15,20 +15,39 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include <string>
-
 #include "arrow/adapters/orc/adapter.h"
-#include "arrow/array.h"
-#include "arrow/io/api.h"
 
 #include <gtest/gtest.h>
+
 #include <orc/OrcFile.hh>
+#include <string>
+
+#include "arrow/adapters/orc/adapter_util.h"
+#include "arrow/api.h"
+#include "arrow/array.h"
+#include "arrow/io/api.h"
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/testing/gtest_util.h"
+#include "arrow/testing/random.h"
+#include "arrow/type.h"
+#include "arrow/util/decimal.h"
 
 namespace liborc = orc;
 
 namespace arrow {
 
-constexpr int DEFAULT_MEM_STREAM_SIZE = 100 * 1024 * 1024;
+constexpr int kDefaultSmallMemStreamSize = 16384 * 5;  // 80KB
+constexpr int kDefaultMemStreamSize = 10 * 1024 * 1024;
+static constexpr random::SeedType kRandomSeed = 0x0ff1ce;
+
+using ArrayBuilderVector = std::vector<std::shared_ptr<ArrayBuilder>>;

Review comment:
       I think these aliases would probably be less common if the comments below about reduce test size are used.  My preference would be to not use them (we use such aliases sparingly in other parts of the code.)




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614598341



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
       On the ORC error issue..let me repeat it here:
   
   @emkornfield @pitrou It does actually look like IOError in this case. In the ORC reader liborc::ParseError is converted into IOError for IO errors and Invalid for other errors. Since this PR does not include fixing all issues in the ORC reader (which I would like to do in a latter PR after my current issues have been addressed) and that in the ORC writer there is only one instance of liborc::ParseError my current fix is adding the other two catches. However if you guys insist I can revamp error processing in the ORC reader tomorrow.




-- 
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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-822319363


   I've made a bunch of fixes (including restoring the Python integration tests). I'll merge if CI is green.


-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r563393105



##########
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##########
@@ -157,4 +249,2151 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
     EXPECT_TRUE(stripe_reader->ReadNext(&record_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// General
+TEST(TestAdapterWriteGeneral, writeZeroRows) {
+  std::vector<std::shared_ptr<Field>> xFields{field("bool", boolean()),
+                                              field("int8", int8()),
+                                              field("int16", int16()),
+                                              field("int32", int32()),
+                                              field("int64", int64()),
+                                              field("float", float32()),
+                                              field("double", float64()),
+                                              field("decimal128nz", decimal(25, 6)),
+                                              field("decimal128z", decimal(32, 0)),
+                                              field("date32", date32()),
+                                              field("ts3", timestamp(TimeUnit::NANO)),
+                                              field("string", utf8()),
+                                              field("binary", binary())};
+  std::shared_ptr<Schema> sharedPtrSchema = std::make_shared<Schema>(xFields);
+
+  int64_t numCols = xFields.size();
+
+  ArrayBuilderVector builders(numCols, NULLPTR);
+  builders[0] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BooleanBuilder>());
+  builders[1] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int8Builder>());
+  builders[2] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int16Builder>());
+  builders[3] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int32Builder>());
+  builders[4] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Int64Builder>());
+  builders[5] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<FloatBuilder>());
+  builders[6] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<DoubleBuilder>());
+  builders[7] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(25, 6)));
+  builders[8] = std::static_pointer_cast<ArrayBuilder>(
+      std::make_shared<Decimal128Builder>(decimal(32, 0)));
+  builders[9] = std::static_pointer_cast<ArrayBuilder>(std::make_shared<Date32Builder>());
+  builders[10] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<TimestampBuilder>(
+          timestamp(TimeUnit::NANO), default_memory_pool()));
+  builders[11] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<StringBuilder>());
+  builders[12] =
+      std::static_pointer_cast<ArrayBuilder>(std::make_shared<BinaryBuilder>());
+  ArrayVector arrays(numCols, NULLPTR);
+  ChunkedArrayVector cv;
+  cv.reserve(numCols);
+
+  for (int col = 0; col < numCols; col++) {
+    ARROW_EXPECT_OK(builders[col]->Finish(&arrays[col]));
+    cv.push_back(std::make_shared<ChunkedArray>(arrays[col]));

Review comment:
       Done! Thank! `TableFromJSON` is very useful for empty table 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962439



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +476,102 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(arrow::io::OutputStream& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return kOrcNaturalWriteSize; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_.Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_.closed()) {
+      ORC_THROW_NOT_OK(output_stream_.Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  arrow::io::OutputStream& output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(arrow::io::OutputStream* output_stream) {
+    out_stream_ = std::unique_ptr<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(*output_stream)));
+    return Status::OK();
+  }
+  Status Write(const Table& table) {
+    std::unique_ptr<liborc::WriterOptions> orc_options =
+        std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    std::unique_ptr<liborc::Type> orc_schema = GetORCType(*(table.schema())).ValueOrDie();
+    try {
+      writer_ = createWriter(*orc_schema, out_stream_.get(), *orc_options);

Review comment:
       Yup it is another ORC function. 




-- 
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614416127



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow

Review comment:
       Ah...thanks! 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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-775048759


   @mathyingzhou Feel free to ping me when this is ready for review again!


----------------------------------------------------------------
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.

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



[GitHub] [arrow] pitrou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
pitrou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-790619135


   > Shall random decimal128 generation with precision > 18 be impossible? It used to be perfectly fine.
   
   Did it actually work? I think it would use random fixed size binary generation, which wouldn't guarantee the generated values fit in the given precision...
   


----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou edited a comment on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou edited a comment on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-792154086


   @pitrou I have found the cause of the problem which is me failing to account for the possibility of orc::Decimal64VectorBatch. I will edit my writer to account for that.
   
   After this PR I'd like to work with you on adapter/testing/random. As you said before tests need to be added. Moreover we should expand random things to RecordBatches, ChunkedArrays & Tables as well as all types. After that we should simplify ORC, Parquet and other 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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562260809



##########
File path: cpp/src/arrow/adapters/orc/adapter.cc
##########
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr<io::OutputStream>& output_stream)
+      : output_stream_(output_stream), length_(0) {}
+
+  uint64_t getLength() const override { return length_; }
+
+  uint64_t getNaturalWriteSize() const override { return ORC_NATURAL_WRITE_SIZE; }
+
+  void write(const void* buf, size_t length) override {
+    ORC_THROW_NOT_OK(output_stream_->Write(buf, static_cast<int64_t>(length)));
+    length_ += static_cast<int64_t>(length);
+  }
+
+  const std::string& getName() const override {
+    static const std::string filename("ArrowOutputFile");
+    return filename;
+  }
+
+  void close() override {
+    if (!output_stream_->closed()) {
+      ORC_THROW_NOT_OK(output_stream_->Close());
+    }
+  }
+
+  void set_length(int64_t length) { length_ = length; }
+
+ private:
+  std::shared_ptr<io::OutputStream> output_stream_;
+  int64_t length_;
+};
+
+class ORCFileWriter::Impl {
+ public:
+  Status Open(const std::shared_ptr<Schema>& schema,
+              const std::shared_ptr<io::OutputStream>& output_stream) {
+    orc_options_ = std::unique_ptr<liborc::WriterOptions>(new liborc::WriterOptions());
+    outStream_ = ORC_UNIQUE_PTR<liborc::OutputStream>(
+        static_cast<liborc::OutputStream*>(new ArrowOutputStream(output_stream)));
+    ORC_THROW_NOT_OK(GetORCType(*schema, &orcSchema_));
+    try {
+      writer_ = createWriter(*orcSchema_, outStream_.get(), *orc_options_);
+    } catch (const liborc::ParseError& e) {
+      return Status::IOError(e.what());
+    }
+    schema_ = schema;
+    num_cols_ = schema->num_fields();
+    return Status::OK();
+  }
+  Status Write(const std::shared_ptr<Table> table) {
+    int64_t numRows = table->num_rows();
+    int64_t batch_size = 1024;  // Doesn't matter what it is
+    std::vector<int64_t> arrowIndexOffset(num_cols_, 0);
+    std::vector<int> arrowChunkOffset(num_cols_, 0);
+    ORC_UNIQUE_PTR<liborc::ColumnVectorBatch> batch = writer_->createRowBatch(batch_size);
+    liborc::StructVectorBatch* root =
+        internal::checked_cast<liborc::StructVectorBatch*>(batch.get());
+    std::vector<liborc::ColumnVectorBatch*> fields = root->fields;
+    while (numRows > 0) {
+      for (int i = 0; i < num_cols_; i++) {
+        ORC_THROW_NOT_OK(adapters::orc::FillBatch(
+            schema_->field(i)->type().get(), fields[i], arrowIndexOffset[i],
+            arrowChunkOffset[i], batch_size, table->column(i).get()));

Review comment:
       This is intentional. Multiple ORC batches and multiple Arrow chunks are expected.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#issuecomment-820963616


   @pitrou Really thanks for your detailed comments! I have addressed all of them. Please review again since we need to release it. Thanks!
   
   


-- 
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.

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



[GitHub] [arrow] xhochy commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
xhochy commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r562161018



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -316,10 +326,482 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
   }
 }
 
+template <class array_type, class batch_type>
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class batch_type, class target_type>
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                            int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                            Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<batch_type*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->data[orcOffset] = static_cast<target_type>(array->Value(arrowOffset));
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillDate64Batch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Date64Array*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t miliseconds = array->Value(arrowOffset);
+      batch->data[orcOffset] =
+          static_cast<int64_t>(std::floor(miliseconds / kOneSecondMillis));
+      batch->nanoseconds[orcOffset] =
+          (miliseconds - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillTimestampBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                          int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                          Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<TimestampArray*>(parray);
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      int64_t data = array->Value(arrowOffset);
+      batch->notNull[orcOffset] = true;
+      switch (std::static_pointer_cast<TimestampType>(array->type())->unit()) {
+        case TimeUnit::type::SECOND: {
+          batch->data[orcOffset] = data;
+          batch->nanoseconds[orcOffset] = 0;
+          break;
+        }
+        case TimeUnit::type::MILLI: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMillis));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMillis * batch->data[orcOffset]) * kOneMilliNanos;
+          break;
+        }
+        case TimeUnit::type::MICRO: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondMicros));
+          batch->nanoseconds[orcOffset] =
+              (data - kOneSecondMicros * batch->data[orcOffset]) * kOneMicroNanos;
+          break;
+        }
+        default: {
+          batch->data[orcOffset] =
+              static_cast<int64_t>(std::floor(data / kOneSecondNanos));
+          batch->nanoseconds[orcOffset] = data - kOneSecondNanos * batch->data[orcOffset];
+        }
+      }
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillStringBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      std::string dataString = array->GetString(arrowOffset);
+      int dataStringLength = dataString.length();
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataStringLength + 1];  // Include null
+      memcpy(batch->data[orcOffset], dataString.c_str(), dataStringLength + 1);
+      batch->length[orcOffset] = dataStringLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+template <class array_type, class offset_type>
+Status FillBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      offset_type dataLength = 0;
+      const uint8_t* data = array->GetValue(arrowOffset, &dataLength);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[dataLength];  // Do not include null
+      memcpy(batch->data[orcOffset], data, dataLength);
+      batch->length[orcOffset] = dataLength;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeBinaryBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                                int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                                Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeBinaryArray*>(parray);
+  auto batch = checked_cast<liborc::StringVectorBatch*>(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int32_t byteWidth = array->byte_width();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      const uint8_t* data = array->GetValue(arrowOffset);
+      if (batch->data[orcOffset]) delete batch->data[orcOffset];
+      batch->data[orcOffset] = new char[byteWidth];  // Do not include null
+      memcpy(batch->data[orcOffset], data, byteWidth);
+      batch->length[orcOffset] = byteWidth;
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+Status FillDecimalBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                        int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                        Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<Decimal128Array*>(parray);
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(cbatch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+      uint8_t* rawInt128 = const_cast<uint8_t*>(array->GetValue(arrowOffset));
+      uint64_t* lowerBits = reinterpret_cast<uint64_t*>(rawInt128);
+      int64_t* higherBits = reinterpret_cast<int64_t*>(rawInt128 + 8);
+      batch->values[orcOffset] = liborc::Int128(*higherBits, *lowerBits);
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillStructBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                       int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                       Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<StructArray*>(parray);
+  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+  std::shared_ptr<std::vector<bool>> outgoingMask;
+  std::size_t size = type->fields().size();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  int64_t initORCOffset = orcOffset;
+  int64_t initArrowOffset = arrowOffset;
+  // First fill fields of ColumnVectorBatch
+  if (array->null_count() || incomingMask) {
+    batch->hasNulls = true;
+    outgoingMask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoingMask = NULLPTR;
+  }
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      (*outgoingMask)[orcOffset] = false;
+    } else {
+      batch->notNull[orcOffset] = true;
+    }
+  }
+  batch->numElements += orcOffset - initORCOffset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    orcOffset = initORCOffset;
+    arrowOffset = initArrowOffset;
+    RETURN_NOT_OK(FillBatch(type->field(i)->type().get(), batch->fields[i], arrowOffset,
+                            orcOffset, length, array->field(i).get(),
+                            outgoingMask.get()));
+  }
+  return Status::OK();
+}
+
+template <class array_type>
+Status FillListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                     int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                     Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<array_type*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      elementBatch->resize(batch->offsets[orcOffset + 1]);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillFixedSizeListBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                              int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                              Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<FixedSizeListArray*>(parray);
+  auto batch = checked_cast<liborc::ListVectorBatch*>(cbatch);
+  auto elementBatch = (batch->elements).get();
+  DataType* elementType = array->value_type().get();
+  int64_t arrowLength = array->length();
+  int32_t elementLength = array->value_length();  // Fixed length of each subarray
+  if (!arrowLength) return Status::OK();
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] + elementLength;
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1];
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, array->values().get(),
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillMapBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                    int64_t& arrowOffset, int64_t& orcOffset, int64_t length,
+                    Array* parray, std::vector<bool>* incomingMask) {
+  auto array = checked_cast<MapArray*>(parray);
+  auto batch = checked_cast<liborc::MapVectorBatch*>(cbatch);
+  auto keyBatch = (batch->keys).get();
+  auto elementBatch = (batch->elements).get();
+  auto keyArray = array->keys().get();
+  auto elementArray = array->items().get();
+  DataType* keyType = keyArray->type().get();
+  DataType* elementType = elementArray->type().get();
+  int64_t arrowLength = array->length();
+  if (!arrowLength) return Status::OK();
+  // int64_t initORCOffset = orcOffset, initArrowOffset = arrowOffset;
+  if (orcOffset == 0) batch->offsets[0] = 0;
+  if (array->null_count() || incomingMask) batch->hasNulls = true;
+  for (; orcOffset < length && arrowOffset < arrowLength; orcOffset++, arrowOffset++) {
+    if (array->IsNull(arrowOffset) || (incomingMask && !(*incomingMask)[orcOffset])) {
+      batch->notNull[orcOffset] = false;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset];
+    } else {
+      batch->notNull[orcOffset] = true;
+      batch->offsets[orcOffset + 1] = batch->offsets[orcOffset] +
+                                      array->value_offset(arrowOffset + 1) -
+                                      array->value_offset(arrowOffset);
+      int64_t subarrayArrowOffset = array->value_offset(arrowOffset),
+              subarrayORCOffset = batch->offsets[orcOffset],
+              subarrayORCLength = batch->offsets[orcOffset + 1],
+              initSubarrayArrowOffset = subarrayArrowOffset,
+              initSubarrayORCOffset = subarrayORCOffset;
+      keyBatch->resize(subarrayORCLength);
+      elementBatch->resize(subarrayORCLength);
+      RETURN_NOT_OK(FillBatch(keyType, keyBatch, subarrayArrowOffset, subarrayORCOffset,
+                              subarrayORCLength, keyArray, NULLPTR));
+      subarrayArrowOffset = initSubarrayArrowOffset;
+      subarrayORCOffset = initSubarrayORCOffset;
+      RETURN_NOT_OK(FillBatch(elementType, elementBatch, subarrayArrowOffset,
+                              subarrayORCOffset, subarrayORCLength, elementArray,
+                              NULLPTR));
+    }
+  }
+  batch->numElements = orcOffset;
+  return Status::OK();
+}
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+                 int64_t& arrowOffset, int64_t& orcOffset, int64_t length, Array* parray,
+                 std::vector<bool>* incomingMask) {

Review comment:
       Please add this as a comment somewhere in the code, I couldn't easily figure it out by reading the code initially. With this comment it makes sense directly.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567593393



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->data[*orc_offset] =
+          static_cast<TargetType>(numeric_array.Value(*arrow_offset));
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteTimestampBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                  int64_t* arrow_offset, int64_t* orc_offset,
+                                  const int64_t& length, const arrow::Array& array,
+                                  const std::vector<bool>* incoming_mask,
+                                  const int64_t& conversion_factor_from_second,
+                                  const int64_t& conversion_factor_to_nano) {
+  const ArrayType& timestamp_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::TimestampVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      int64_t data = timestamp_array.Value(*arrow_offset);
+      batch->notNull[*orc_offset] = true;
+      batch->data[*orc_offset] =
+          static_cast<int64_t>(std::floor(data / conversion_factor_from_second));
+      batch->nanoseconds[*orc_offset] =
+          (data - conversion_factor_from_second * batch->data[*orc_offset]) *
+          conversion_factor_to_nano;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+template <class ArrayType, class OffsetType>
+arrow::Status WriteBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const ArrayType& binary_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      OffsetType data_length = 0;
+      const uint8_t* data = binary_array.GetValue(*arrow_offset, &data_length);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteFixedSizeBinaryBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                        int64_t* arrow_offset, int64_t* orc_offset,
+                                        const int64_t& length, const arrow::Array& array,
+                                        const std::vector<bool>* incoming_mask) {
+  const arrow::FixedSizeBinaryArray& fixed_size_binary_array(
+      checked_cast<const arrow::FixedSizeBinaryArray&>(array));
+  auto batch = checked_cast<liborc::StringVectorBatch*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int32_t data_length = fixed_size_binary_array.byte_width();
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      const uint8_t* data = fixed_size_binary_array.GetValue(*arrow_offset);
+      if (batch->data[*orc_offset]) delete batch->data[*orc_offset];
+      batch->data[*orc_offset] = new char[data_length];  // Do not include null
+      memcpy(batch->data[*orc_offset], data, data_length);
+      batch->length[*orc_offset] = data_length;
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+// If Arrow supports 256-bit decimals we can not support it unless ORC does it
+arrow::Status WriteDecimalBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const arrow::Decimal128Array& decimal128_array(
+      checked_cast<const arrow::Decimal128Array&>(array));
+  auto batch = checked_cast<liborc::Decimal128VectorBatch*>(column_vector_batch);
+  // Arrow uses 128 bits for decimal type and in the future, 256 bits will also be
+  // supported.
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+      uint8_t* raw_int128 =
+          const_cast<uint8_t*>(decimal128_array.GetValue(*arrow_offset));
+      uint64_t* lower_bits = reinterpret_cast<uint64_t*>(raw_int128);
+      int64_t* higher_bits = reinterpret_cast<int64_t*>(raw_int128 + 8);
+      batch->values[*orc_offset] = liborc::Int128(*higher_bits, *lower_bits);
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteStructBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                               int64_t* arrow_offset, int64_t* orc_offset,
+                               const int64_t& length, const arrow::Array& array,
+                               const std::vector<bool>* incoming_mask) {
+  const arrow::StructArray& struct_array(checked_cast<const arrow::StructArray&>(array));
+  auto batch = checked_cast<liborc::StructVectorBatch*>(column_vector_batch);
+  std::shared_ptr<std::vector<bool>> outgoing_mask;
+  std::size_t size = array.type()->fields().size();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  const int64_t init_orc_offset = *orc_offset;
+  const int64_t init_arrow_offset = *arrow_offset;
+  // First fill fields of ColumnVectorBatch
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+    outgoing_mask = std::make_shared<std::vector<bool>>(length, true);
+  } else {
+    outgoing_mask = nullptr;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      (*outgoing_mask)[*orc_offset] = false;
+    } else {
+      batch->notNull[*orc_offset] = true;
+    }
+  }
+  batch->numElements += *orc_offset - init_orc_offset;
+  // Fill the fields
+  for (std::size_t i = 0; i < size; i++) {
+    *orc_offset = init_orc_offset;
+    *arrow_offset = init_arrow_offset;
+    RETURN_NOT_OK(WriteBatch(batch->fields[i], arrow_offset, orc_offset, length,
+                             *(struct_array.field(i)), outgoing_mask.get()));
+  }
+  return arrow::Status::OK();
+}
+
+template <class ArrayType>
+arrow::Status WriteListBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                             int64_t* arrow_offset, int64_t* orc_offset,
+                             const int64_t& length, const arrow::Array& array,
+                             const std::vector<bool>* incoming_mask) {
+  const ArrayType& list_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<liborc::ListVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        list_array.value_offset(*arrow_offset + 1) -
+                                        list_array.value_offset(*arrow_offset);
+      element_batch->resize(batch->offsets[*orc_offset + 1]);
+      int64_t subarray_arrow_offset = list_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1];
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length,
+                               *(list_array.values()), nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteMapBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                            int64_t* arrow_offset, int64_t* orc_offset,
+                            const int64_t& length, const arrow::Array& array,
+                            const std::vector<bool>* incoming_mask) {
+  const arrow::MapArray& map_array(checked_cast<const arrow::MapArray&>(array));
+  auto batch = checked_cast<liborc::MapVectorBatch*>(column_vector_batch);
+  liborc::ColumnVectorBatch* key_batch = (batch->keys).get();
+  liborc::ColumnVectorBatch* element_batch = (batch->elements).get();
+  std::shared_ptr<arrow::Array> key_array = map_array.keys();
+  std::shared_ptr<arrow::Array> element_array = map_array.items();
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (*orc_offset == 0) {
+    batch->offsets[0] = 0;
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||
+        (incoming_mask && !(*incoming_mask)[*orc_offset])) {
+      batch->notNull[*orc_offset] = false;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset];
+    } else {
+      batch->notNull[*orc_offset] = true;
+      batch->offsets[*orc_offset + 1] = batch->offsets[*orc_offset] +
+                                        map_array.value_offset(*arrow_offset + 1) -
+                                        map_array.value_offset(*arrow_offset);
+      int64_t subarray_arrow_offset = map_array.value_offset(*arrow_offset),
+              subarray_orc_offset = batch->offsets[*orc_offset],
+              subarray_orc_length = batch->offsets[*orc_offset + 1],
+              init_subarray_arrow_offset = subarray_arrow_offset,
+              init_subarray_orc_offset = subarray_orc_offset;
+      key_batch->resize(subarray_orc_length);
+      element_batch->resize(subarray_orc_length);
+      RETURN_NOT_OK(WriteBatch(key_batch, &subarray_arrow_offset, &subarray_orc_offset,
+                               subarray_orc_length, *key_array, nullptr));
+      subarray_arrow_offset = init_subarray_arrow_offset;
+      subarray_orc_offset = init_subarray_orc_offset;
+      RETURN_NOT_OK(WriteBatch(element_batch, &subarray_arrow_offset,
+                               &subarray_orc_offset, subarray_orc_length, *element_array,
+                               nullptr));
+    }
+  }
+  batch->numElements = *orc_offset;
+  return arrow::Status::OK();
+}
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& array,
+                         const std::vector<bool>* incoming_mask) {
+  arrow::Type::type kind = array.type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+      return WriteNumericBatch<arrow::BooleanArray, liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT8:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int8Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT16:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int16Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::INT64:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Int64Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FLOAT:
+      return WriteNumericBatch<arrow::NumericArray<arrow::FloatType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DOUBLE:
+      return WriteNumericBatch<arrow::NumericArray<arrow::DoubleType>,
+                               liborc::DoubleVectorBatch, double>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::BINARY:
+      return WriteBinaryBatch<arrow::BinaryArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_BINARY:
+      return WriteBinaryBatch<arrow::LargeBinaryArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::STRING:
+      return WriteBinaryBatch<arrow::StringArray, int32_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::LARGE_STRING:
+      return WriteBinaryBatch<arrow::LargeStringArray, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+      return WriteFixedSizeBinaryBatch(column_vector_batch, arrow_offset, orc_offset,
+                                       length, array, incoming_mask);
+    case arrow::Type::type::DATE32:
+      return WriteNumericBatch<arrow::NumericArray<arrow::Date32Type>,
+                               liborc::LongVectorBatch, int64_t>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask);
+    case arrow::Type::type::DATE64:
+      return WriteTimestampBatch<arrow::Date64Array>(
+          column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+          kOneSecondMillis, kOneMilliNanos);
+    case arrow::Type::type::TIMESTAMP: {
+      switch (arrow::internal::checked_pointer_cast<arrow::TimestampType>(array.type())
+                  ->unit()) {
+        case arrow::TimeUnit::type::SECOND:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              1, kOneSecondNanos);
+        case arrow::TimeUnit::type::MILLI:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMillis, kOneMilliNanos);
+        case arrow::TimeUnit::type::MICRO:
+          return WriteTimestampBatch<arrow::TimestampArray>(
+              column_vector_batch, arrow_offset, orc_offset, length, array, incoming_mask,
+              kOneSecondMicros, kOneMicroNanos);
+        default:

Review comment:
       it is probably not a good idea of have an implicit default here (you should spell out nanos and default should be returning an error.




----------------------------------------------------------------
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567586684



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +342,462 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                         int64_t* arrow_offset, int64_t* orc_offset,
+                         const int64_t& length, const arrow::Array& parray,
+                         const std::vector<bool>* incoming_mask = NULLPTR);
+
+// incoming_mask is exclusively used by FillStructBatch. The cause is that ORC is much
+// stricter than Arrow in terms of consistency. In this case if a struct scalar is null
+// all its children must be set to null or ORC is not going to function properly. This is
+// why I added incoming_mask to pass on null status from a struct to its children.
+//
+// static_cast from int64_t or double to itself shouldn't introduce overhead
+// Pleae see
+// https://stackoverflow.com/questions/19106826/
+// can-static-cast-to-same-type-introduce-runtime-overhead
+template <class ArrayType, class BatchType, class TargetType>
+arrow::Status WriteNumericBatch(liborc::ColumnVectorBatch* column_vector_batch,
+                                int64_t* arrow_offset, int64_t* orc_offset,
+                                const int64_t& length, const arrow::Array& array,
+                                const std::vector<bool>* incoming_mask) {
+  const ArrayType& numeric_array(checked_cast<const ArrayType&>(array));
+  auto batch = checked_cast<BatchType*>(column_vector_batch);
+  int64_t arrow_length = array.length();
+  if (!arrow_length) {
+    return arrow::Status::OK();
+  }
+  if (array.null_count() || incoming_mask) {
+    batch->hasNulls = true;
+  }
+  for (; *orc_offset < length && *arrow_offset < arrow_length;
+       (*orc_offset)++, (*arrow_offset)++) {
+    if (array.IsNull(*arrow_offset) ||

Review comment:
       Actually, looking at this and subsequent code.  I think the clearest thing that you can do construct a new Array if necessary using the newly computing bit mask and rewriting all of these Write*Batch to use ArrayDataInlineVisitor




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614152373



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include <cmath>
 #include <string>
 #include <vector>
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 1000000LL;
+constexpr int64_t kOneMilliNanos = 1000000LL;
+constexpr int64_t kOneSecondNanos = 1000000000LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {
+namespace {
 
-using internal::checked_cast;
+using arrow::internal::checked_cast;
 
-// The number of nanoseconds in a second
-constexpr int64_t kOneSecondNanos = 1000000000LL;
-
-Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* cbatch,
-                         int64_t offset, int64_t length, ArrayBuilder* abuilder) {
-  auto builder = checked_cast<StructBuilder*>(abuilder);
-  auto batch = checked_cast<liborc::StructVectorBatch*>(cbatch);
+arrow::Status AppendStructBatch(const liborc::Type* type,

Review comment:
       Ah I put them in the anonymous namespace so it was actually a nontrivial change.




-- 
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.

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



[GitHub] [arrow] emkornfield commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
emkornfield commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567580521



##########
File path: cpp/src/arrow/adapters/orc/adapter.h
##########
@@ -142,6 +143,36 @@ class ARROW_EXPORT ORCFileReader {
   ORCFileReader();
 };
 
+/// \class ORCFileWriter
+/// \brief Write an Arrow Table or RecordBatch to an ORC file.
+class ARROW_EXPORT ORCFileWriter {
+ public:
+  ~ORCFileWriter();
+  /// \brief Creates a new ORC writer.
+  ///
+  /// \param[in] output_stream the io::OutputStream to write into
+  /// \return the returned writer object
+  static Result<std::unique_ptr<ORCFileWriter>> Open(io::OutputStream& output_stream);

Review comment:
       Please add documentation to clarify ownership semantics (and who needs to call close on output_stream




----------------------------------------------------------------
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.

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



[GitHub] [arrow] mathyingzhou commented on a change in pull request #8648: ARROW-7906: [C++] [Python] Add ORC write support

Posted by GitBox <gi...@apache.org>.
mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614497672



##########
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##########
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, liborc::ColumnVectorBatch* batch,
       return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+                         liborc::ColumnVectorBatch* column_vector_batch,
+                         bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr<arrow::Array> NormalizeArray(const std::shared_ptr<arrow::Array>& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+    case arrow::Type::type::BOOL:
+    case arrow::Type::type::INT8:
+    case arrow::Type::type::INT16:
+    case arrow::Type::type::INT32:
+    case arrow::Type::type::INT64:
+    case arrow::Type::type::FLOAT:
+    case arrow::Type::type::DOUBLE:
+    case arrow::Type::type::STRING:
+    case arrow::Type::type::LARGE_STRING:
+    case arrow::Type::type::BINARY:
+    case arrow::Type::type::LARGE_BINARY:
+    case arrow::Type::type::FIXED_SIZE_BINARY:
+    case arrow::Type::type::DATE32:
+    case arrow::Type::type::DATE64:
+    case arrow::Type::type::TIMESTAMP:
+    case arrow::Type::type::DECIMAL128: {

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.

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