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 2021/01/21 16:40:44 UTC

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

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