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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614747421



##
File path: python/pyarrow/_orc.pyx
##
@@ -109,3 +113,28 @@ cdef class ORCReader(_Weakrefable):
 check_status(deref(self.reader).Read(indices, _table))
 
 return pyarrow_wrap_table(sp_table)
+
+cdef class ORCWriter(_Weakrefable):
+cdef:
+object source
+unique_ptr[ORCFileWriter] writer
+
+def open(self, object source):
+cdef:
+shared_ptr[COutputStream] rd_handle
+self.source = source
+get_writer(source, _handle)
+with nogil:
+self.writer = move(GetResultValue[unique_ptr[ORCFileWriter]](
+ORCFileWriter.Open(rd_handle.get(

Review comment:
   Done.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



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

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




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614743037



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

Review comment:
   @pitrou Fair enough. Well, back then in Dec and Jan I did manual tests 
using the ORC adapter to write ORC files and then loaded them using pyorc and 
compared the results. So we should be good.
   
   Of course in the future I can add some more tests (read using Arrow, write 
without Arrow and vice versa). Can we get this PR out there though? The 
functionality has been very stable since early Jan and the amount of bugs 
affecting the actual ORC files we have caught during the past 3 months is 2-4.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614743037



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

Review comment:
   @pitrou Fair enough. Well, back then in Dec and Jan I did manual tests 
using the ORC adapter to write ORC files and then loaded them using pyorc and 
compared the results. We should be good.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614620841



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

Review comment:
   Well, I removed a test that compares the ORC schema with some JSON one 
since they don’t actually line up given the new behavior of the ORC Reader on 
MAP arrays.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614606429



##
File path: python/pyarrow/_orc.pyx
##
@@ -109,3 +113,28 @@ cdef class ORCReader(_Weakrefable):
 check_status(deref(self.reader).Read(indices, _table))
 
 return pyarrow_wrap_table(sp_table)
+
+cdef class ORCWriter(_Weakrefable):
+cdef:
+object source
+unique_ptr[ORCFileWriter] writer
+
+def open(self, object source):
+cdef:
+shared_ptr[COutputStream] rd_handle
+self.source = source
+get_writer(source, _handle)
+with nogil:
+self.writer = move(GetResultValue[unique_ptr[ORCFileWriter]](
+ORCFileWriter.Open(rd_handle.get(

Review comment:
   Thanks! Given that we use the `pImpl` pattern for `ORCFileWriter` (and 
in fact `ORCFileReader` as well) I will store it in the Python `ORCWriter` 
object.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



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

Review comment:
   It closes the `std::unique_ptr writer_` so yes closure 
does take place. Doc clarified.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614603354



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

Review comment:
   Restored.




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614598341



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

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




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614598009



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

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




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

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




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

2021-04-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614590764



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614575581



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
   On `WriteBatch`: Doc added and parameters rotated so that they make more 
sense.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614561787



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

Review comment:
   Wait. This function itself actually contains ORC-specific code such as 
the requirement that Date64 and Timestamp scalars must not overflow when cast 
to Timestamp NANO. Unless this requirement is actually universal in which case 
we should change how random arrays are canonically generated for these types we 
shouldn't really leave some function so ORC-specific in `testing/random.h`.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614561787



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614558933



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614556529



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614556421



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



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

Review comment:
   Thanks! Ah actually we don't need it for this release since we don't 
support NA (ORC doesn't have it) or union types. I do need to add it when 
implementing the ORC reader & writer for union types though. 




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



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

Review comment:
   Ah actually we don't need it for this release since we don't support NA 
or union types. I need to add it when implementing the ORC reader & writer for 
union types though. 




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614551452



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

Review comment:
   Ah actually we don't need it for this release. I need to add it when 
implementing the ORC reader & writer for union types though. 




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614550532



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

Review comment:
   Hmm this breaks the Struct of Struct test. There must be some subtle 
difference between the two.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614527666



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614523227



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614523227



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614522859



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614522411



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614516746



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

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

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

Review comment:
   Yup.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614516188



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614515778



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614513426



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512702



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512702



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512116



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614512073



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614511596



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614510493



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614510493



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614508964



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow
+
+namespace {
+
+using arrow::internal::checked_cast;
+
+arrow::Status WriteBatch(const arrow::Array& parray, int64_t orc_offset,
+ liborc::ColumnVectorBatch* column_vector_batch,
+ bool normalized = false);
+
+// Make sure children of StructArray have appropriate null.
+std::shared_ptr NormalizeArray(const 
std::shared_ptr& array) {
+  arrow::Type::type kind = array->type_id();
+  switch (kind) {
+case arrow::Type::type::BOOL:
+case arrow::Type::type::INT8:
+case arrow::Type::type::INT16:
+case arrow::Type::type::INT32:
+case arrow::Type::type::INT64:
+case arrow::Type::type::FLOAT:
+case arrow::Type::type::DOUBLE:
+case arrow::Type::type::STRING:
+case arrow::Type::type::LARGE_STRING:
+case arrow::Type::type::BINARY:
+case arrow::Type::type::LARGE_BINARY:
+case arrow::Type::type::FIXED_SIZE_BINARY:
+case arrow::Type::type::DATE32:
+case arrow::Type::type::DATE64:
+case arrow::Type::type::TIMESTAMP:
+case arrow::Type::type::DECIMAL128: {
+  return array;
+}
+case arrow::Type::type::STRUCT: {
+  if (array->null_count() == 0) {
+return array;
+  } else {
+auto struct_array = 
std::static_pointer_cast(array);
+const std::shared_ptr bitmap = 
struct_array->null_bitmap();
+std::shared_ptr struct_type = struct_array->type();
+std::size_t size = struct_type->fields().size();
+std::vector> new_children(size, nullptr);
+for (std::size_t i = 0; i < size; i++) {
+  std::shared_ptr child = struct_array->field(i);
+  const std::shared_ptr child_bitmap = 
child->null_bitmap();
+  std::shared_ptr final_child_bitmap;
+  if (child_bitmap == nullptr) {
+final_child_bitmap = bitmap;
+  } else {
+final_child_bitmap = arrow::internal::BitmapAnd(
+ arrow::default_memory_pool(), 
bitmap->data(), 0,
+ child_bitmap->data(), 0, 
struct_array->length(), 0)
+ .ValueOrDie();
+  }
+  std::shared_ptr child_array_data = child->data();
+  std::vector> child_buffers =
+  child_array_data->buffers;
+  child_buffers[0] = final_child_bitmap;
+  std::shared_ptr new_child_array_data = 
arrow::ArrayData::Make(
+  child->type(), child->length(), child_buffers, 
child_array_data->child_data,
+  child_array_data->dictionary);
+  new_children[i] = 
NormalizeArray(arrow::MakeArray(new_child_array_data));
+}
+return std::make_shared(struct_type, 
struct_array->length(),
+new_children, bitmap);
+  }
+}
+case arrow::Type::type::LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::LARGE_LIST: {
+  auto list_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
list_array->value_offsets(),
+  NormalizeArray(list_array->values()), list_array->null_bitmap());
+}
+case arrow::Type::type::FIXED_SIZE_LIST: {
+  auto list_array = 
std::static_pointer_cast(array);
+  return std::make_shared(
+  list_array->type(), list_array->length(), 
NormalizeArray(list_array->values()),
+  list_array->null_bitmap());
+}
+case arrow::Type::type::MAP: {
+  auto map_array = std::static_pointer_cast(array);
+  return std::make_shared(
+  map_array->type(), map_array->length(), map_array->value_offsets(),
+  NormalizeArray(map_array->keys()), 
NormalizeArray(map_array->items()),
+  map_array->null_bitmap());
+}
+default: {
+  return array;
+}
+  }
+}
+
+template 
+struct Appender {};
+
+// Types for long/double-like Appender, that is, numeric, boolean or date32
+template 
+using is_generic_type =
+std::integral_constant::value ||
+ std::is_same::value 
||
+ arrow::is_boolean_type::value>;
+template 
+using enable_if_generic = arrow::enable_if_t::value, R>;
+
+// Number-like
+template 
+struct Appender> {
+  using ArrayType = 

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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614499856



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

Review comment:
   Thanks! Done.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614499015



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

Review comment:
   Done.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614497672



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

Review comment:
   Sure.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614416127



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -315,13 +344,662 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   return Status::NotImplemented("Not implemented type kind: ", kind);
   }
 }
+}  // namespace orc
+}  // namespace adapters
+}  // namespace arrow

Review comment:
   Ah...thanks! Done.
   




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614414590



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
   Hmm what exactly is the issue with ArrowOutputStream? 




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614413650



##
File path: cpp/src/arrow/testing/random.h
##
@@ -310,6 +310,16 @@ class ARROW_TESTING_EXPORT RandomArrayGenerator {
   std::shared_ptr FixedSizeBinary(int64_t size, int32_t byte_width,
  double null_probability = 0);
 
+  // /// \brief Generate a random StructArray
+  // ///
+  // /// \param[in] children Vector of Arrays containing the data for each 
child
+  // /// \param[in] size The size of the generated list array
+  // /// \param[in] null_probability the probability of a list value being null
+  // ///
+  // /// \return a generated Array
+  // std::shared_ptr Struct(const ArrayVector& children, int64_t size,
+  //   double null_probability);
+

Review comment:
   Yup. Done!




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614408484



##
File path: cpp/src/arrow/adapters/orc/adapter.cc
##
@@ -44,20 +45,15 @@
 #include "arrow/util/macros.h"
 #include "arrow/util/range.h"
 #include "arrow/util/visibility.h"
-
 #include "orc/Exceptions.hh"
-#include "orc/OrcFile.hh"
+
+// The following are required by ORC to be uint64_t
+constexpr uint64_t kOrcWriterBatchSize = 128 * 1024;
+constexpr uint64_t kOrcNaturalWriteSize = 128 * 1024;

Review comment:
   Done!




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614405583



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;

Review comment:
   Sure. Done!




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614403349



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {

Review comment:
   Sure.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614158494



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614157196



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614157196



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614156557



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

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




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614156557



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

Review comment:
   It actually does. Arrow has a lot more types than ORC hence 
Arrow2ORC(ORC2Arrow(x)) may not be the same as x.




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

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




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

2021-04-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r614152373



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -15,38 +15,46 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include "arrow/adapters/orc/adapter_util.h"
+
+#include 
 #include 
 #include 
 
-#include "arrow/adapters/orc/adapter_util.h"
 #include "arrow/array/builder_base.h"
 #include "arrow/builder.h"
+#include "arrow/chunked_array.h"
+#include "arrow/scalar.h"
 #include "arrow/status.h"
+#include "arrow/util/bitmap_ops.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/decimal.h"
 #include "arrow/util/range.h"
-
+#include "arrow/util/string_view.h"
+#include "arrow/visitor_inline.h"
 #include "orc/Exceptions.hh"
+#include "orc/MemoryPool.hh"
 #include "orc/OrcFile.hh"
 
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
+constexpr int64_t kOneSecondNanos = 10LL;
 // alias to not interfere with nested orc namespace
 namespace liborc = orc;
 
-namespace arrow {
-
-namespace adapters {
-
-namespace orc {
+namespace {
 
-using internal::checked_cast;
+using arrow::internal::checked_cast;
 
-// The number of nanoseconds in a second
-constexpr int64_t kOneSecondNanos = 10LL;
-
-Status AppendStructBatch(const liborc::Type* type, liborc::ColumnVectorBatch* 
cbatch,
- int64_t offset, int64_t length, ArrayBuilder* 
abuilder) {
-  auto builder = checked_cast(abuilder);
-  auto batch = checked_cast(cbatch);
+arrow::Status AppendStructBatch(const liborc::Type* type,

Review comment:
   Ah I put them in the anonymous namespace so it was actually a nontrivial 
change.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612963356



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

Review comment:
   Sure. Docs Added.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



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

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




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612990896



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

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




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612990896



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

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




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612976502



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

Review comment:
   Sure. Fixed.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612965289



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

Review comment:
   Given that ORC only has these three exposed errors 
https://github.com/apache/orc/blob/master/c%2B%2B/include/orc/Exceptions.hh 
ParseError is still more reasonable compared to NotImplementedYet or 
InvalidArgument.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612965289



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

Review comment:
   Given that ORC only has these three errors 
https://github.com/apache/orc/blob/master/c%2B%2B/include/orc/Exceptions.hh 
ParseError is still more reasonable compared to NotImplementedYet or 
InvalidArgument.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612964375



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

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




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612963356



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

Review comment:
   Sure.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



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

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




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962806



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

Review comment:
   Ah. Yes.




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

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




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

2021-04-14 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612962439



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

Review comment:
   Yup it is another ORC function. 




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

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




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

2021-04-13 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612933793



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

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




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

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




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

2021-04-13 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r612933592



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

Review comment:
   Thanks! Fixed.




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

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




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

2021-03-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r595293956



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

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





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

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




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

2021-03-16 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594889437



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

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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594816142



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

Review comment:
   Sure. Just fixed that one.





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594787933



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

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





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594783196



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

Review comment:
   @pitrou @emkornfield 





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594782962



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

Review comment:
   None of the symbols here is actually exposed to library users. Do we 
need to document these functions?





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594782467



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

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





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594472371



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -316,10 +326,525 @@ Status AppendBatch(const liborc::Type* type, 
liborc::ColumnVectorBatch* batch,
   }
 }
 
+template 
+Status FillNumericBatch(const DataType* type, liborc::ColumnVectorBatch* 
cbatch,
+int64_t& arrowOffset, int64_t& orcOffset, int64_t 
length,
+Array* parray, std::vector* incomingMask) {
+  auto array = checked_cast(parray);
+  auto batch = checked_cast(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+return Status::OK();
+  }
+  if (array->null_count() || 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 
+Status FillNumericBatchCast(const DataType* type, liborc::ColumnVectorBatch* 
cbatch,
+int64_t& arrowOffset, int64_t& orcOffset, int64_t 
length,
+Array* parray, std::vector* incomingMask) {
+  auto array = checked_cast(parray);
+  auto batch = checked_cast(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+return Status::OK();
+  }
+  if (array->null_count() || 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(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* incomingMask) {
+  auto array = checked_cast(parray);
+  auto batch = checked_cast(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+return Status::OK();
+  }
+  if (array->null_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(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* incomingMask) {
+  auto array = checked_cast(parray);
+  auto batch = checked_cast(cbatch);
+  int64_t arrowLength = array->length();
+  if (!arrowLength) {
+return Status::OK();
+  }
+  if (array->null_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(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(std::floor(data / kOneSecondMillis));
+  batch->nanoseconds[orcOffset] =
+  (data - kOneSecondMillis * batch->data[orcOffset]) * 
kOneMilliNanos;
+  break;
+}
+case TimeUnit::type::MICRO: {
+  batch->data[orcOffset] =
+  static_cast(std::floor(data / kOneSecondMicros));
+  batch->nanoseconds[orcOffset] =
+  (data - kOneSecondMicros * batch->data[orcOffset]) * 
kOneMicroNanos;
+  break;
+}
+default: {
+  batch->data[orcOffset] =
+  

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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594465495



##
File path: cpp/src/arrow/adapters/orc/adapter_util.cc
##
@@ -40,15 +44,21 @@ namespace orc {
 
 using internal::checked_cast;
 
-// The number of nanoseconds in a second
+// The number of milliseconds, microseconds and nanoseconds in a second
+constexpr int64_t kOneSecondMillis = 1000LL;
+constexpr int64_t kOneMicroNanos = 1000LL;
+constexpr int64_t kOneSecondMicros = 100LL;
+constexpr int64_t kOneMilliNanos = 100LL;
 constexpr int64_t kOneSecondNanos = 10LL;
+// 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(abuilder);
   auto batch = checked_cast(cbatch);
 
-  const uint8_t* valid_bytes = nullptr;
+  const uint8_t* valid_bytes = NULLPTR;

Review comment:
   Fixed.





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594464546



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

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





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594463095



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

Review comment:
   Done.





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594462966



##
File path: cpp/src/arrow/adapters/orc/adapter.cc
##
@@ -473,6 +473,105 @@ int64_t ORCFileReader::NumberOfStripes() { return 
impl_->NumberOfStripes(); }
 
 int64_t ORCFileReader::NumberOfRows() { return impl_->NumberOfRows(); }
 
+class ArrowOutputStream : public liborc::OutputStream {
+ public:
+  explicit ArrowOutputStream(const std::shared_ptr& 
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(length)));
+length_ += static_cast(length);
+  }
+
+  const std::string& getName() const override {
+static const std::string filename("ArrowOutputFile");

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





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594461270



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

Review comment:
   Done.





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

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




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

2021-03-15 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r594461039



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

Review comment:
   I have fixed that.





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

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




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

2021-03-10 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591678190



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

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





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

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




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

2021-03-10 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r591678190



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

Review comment:
   @pitrou Ah..sure.





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

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




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

2021-02-18 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r578946643



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

Review comment:
   Done!





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

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




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

2021-02-13 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r575733146



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

Review comment:
   Thanks! This has been fixed!





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

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




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

2021-02-13 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r575731092



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

Review comment:
   Thanks! Done!





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

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




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

2021-02-11 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r574939922



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

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

2021-02-11 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r574939867



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

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

2021-01-31 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567596202



##
File path: cpp/src/arrow/adapters/orc/adapter_test.cc
##
@@ -157,4 +197,1930 @@ TEST(TestAdapter, readIntAndStringFileMultipleStripes) {
 EXPECT_TRUE(stripe_reader->ReadNext(_batch).ok());
   }
 }
+
+// WriteORC tests
+
+// Trivial
+TEST(TestAdapterWriteTrivial, writeZeroRowsNoConversion) {
+  std::shared_ptr table = TableFromJSON(
+  schema({field("bool", boolean()), field("int8", int8()), field("int16", 
int16()),
+  field("int32", int32()), field("int64", int64()), field("float", 
float32()),
+  field("double", float64()), field("decimal128nz", decimal(25, 
6)),
+  field("decimal128z", decimal(32, 0)), field("date32", date32()),
+  field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+  field("binary", binary()),
+  field("struct", struct_({field("a", utf8()), field("b", 
int64())})),
+  field("list", list(int32())),
+  field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+  {R"([])"});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessNoConversion) {
+  std::shared_ptr table = TableFromJSON(
+  schema({field("bool", boolean()), field("int8", int8()), field("int16", 
int16()),
+  field("int32", int32()), field("int64", int64()), field("float", 
float32()),
+  field("double", float64()), field("decimal128nz", decimal(25, 
6)),
+  field("decimal128z", decimal(32, 0)), field("date32", date32()),
+  field("ts3", timestamp(TimeUnit::NANO)), field("string", utf8()),
+  field("binary", binary()),
+  field("struct", struct_({field("a", utf8()), field("b", 
int64())})),
+  field("list", list(int32())),
+  field("lsl", list(struct_({field("lsl0", list(int32()))})))}),
+  {});
+  AssertTableWriteReadEqual(table, table, kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeZeroRowsWithConversion) {
+  std::shared_ptr
+  input_table = TableFromJSON(
+  schema({field("date64", date64()), field("ts0", 
timestamp(TimeUnit::SECOND)),
+  field("ts1", timestamp(TimeUnit::MILLI)),
+  field("ts2", timestamp(TimeUnit::MICRO)),
+  field("large_string", large_utf8()),
+  field("large_binary", large_binary()),
+  field("fixed_size_binary0", fixed_size_binary(0)),
+  field("fixed_size_binary", fixed_size_binary(5)),
+  field("large_list", large_list(int32())),
+  field("fixed_size_list", fixed_size_list(int32(), 3)),
+  field("map", map(utf8(), utf8()))}),
+  {R"([])"}),
+  expected_output_table = TableFromJSON(
+  schema({field("date64", timestamp(TimeUnit::NANO)),
+  field("ts0", timestamp(TimeUnit::NANO)),
+  field("ts1", timestamp(TimeUnit::NANO)),
+  field("ts2", timestamp(TimeUnit::NANO)), 
field("large_string", utf8()),
+  field("large_binary", binary()), field("fixed_size_binary0", 
binary()),
+  field("fixed_size_binary", binary()),
+  field("large_list", list(int32())),
+  field("fixed_size_list", list(int32())),
+  field("map",
+list(struct_({field("key", utf8()), field("value", 
utf8())})))}),
+  {R"([])"});
+  AssertTableWriteReadEqual(input_table, expected_output_table,
+kDefaultSmallMemStreamSize / 16);
+}
+TEST(TestAdapterWriteTrivial, writeChunklessWithConversion) {
+  std::shared_ptr
+  input_table = TableFromJSON(
+  schema({field("date64", date64()), field("ts0", 
timestamp(TimeUnit::SECOND)),
+  field("ts1", timestamp(TimeUnit::MILLI)),
+  field("ts2", timestamp(TimeUnit::MICRO)),
+  field("large_string", large_utf8()),
+  field("large_binary", large_binary()),
+  field("fixed_size_binary0", fixed_size_binary(0)),
+  field("fixed_size_binary", fixed_size_binary(5)),
+  field("large_list", large_list(int32())),
+  field("fixed_size_list", fixed_size_list(int32(), 3)),
+  field("map", map(utf8(), utf8()))}),
+  {}),
+  expected_output_table = TableFromJSON(
+  schema({field("date64", timestamp(TimeUnit::NANO)),
+  field("ts0", timestamp(TimeUnit::NANO)),
+  field("ts1", timestamp(TimeUnit::NANO)),
+  field("ts2", timestamp(TimeUnit::NANO)), 
field("large_string", utf8()),
+  field("large_binary", binary()), field("fixed_size_binary0", 
binary()),

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

2021-01-31 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595572



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

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

2021-01-31 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595076



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

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

2021-01-31 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r567595076



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

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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565774069



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

Review comment:
   Done!





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773335



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

Review comment:
   Yup! Done!





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773246



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

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





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565773156



##
File path: cpp/src/arrow/adapters/orc/adapter_util.h
##
@@ -34,8 +34,21 @@ namespace orc {
 
 Status GetArrowType(const liborc::Type* type, std::shared_ptr* out);
 
+Status GetORCType(const DataType& type, ORC_UNIQUE_PTR* out);
+
+Status GetORCType(const Schema& schema, ORC_UNIQUE_PTR* 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* incomingMask = NULLPTR);
+
+Status FillBatch(const DataType* type, liborc::ColumnVectorBatch* cbatch,
+ int64_t& arrowIndexOffset, int& arrowChunkOffset, int64_t 
length,
+ ChunkedArray* pchunkedArray);

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





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772748



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

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





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772573



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

Review comment:
   Thanks! This has been revamped.





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772513



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

Review comment:
   Thanks! Fixed!





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

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




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

2021-01-27 Thread GitBox


mathyingzhou commented on a change in pull request #8648:
URL: https://github.com/apache/arrow/pull/8648#discussion_r565772435



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

Review comment:
   It is actually used in `adapter.cc` and has been moved there.





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

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




  1   2   >