You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2021/07/26 15:28:08 UTC

[GitHub] [arrow] aocsa opened a new pull request #10802: [C++] Implement "drop null" kernels that return array without nulls

aocsa opened a new pull request #10802:
URL: https://github.com/apache/arrow/pull/10802


   WIP 
   
   Note: This can be implemented as a arrow::compute::VectorFunction because the size of the array is changed, so this function is not valid in a SQL-like context
   
   https://issues.apache.org/jira/browse/ARROW-1568?jql=project%20%3D%20ARROW%20AND%20labels%20%3D%20beginner%20AND%20status%20!%3D%20Resolved%20AND%20component%20%3D%20%22C%2B%2B%22


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now

Review comment:
       It is allowed to create a zero-chunk chunked array, so this shouldn't be needed.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,

Review comment:
       Why don't you use return `Result<...>` in these helper functions?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {

Review comment:
       Here as well, you can use `column->type()->id()`.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());

Review comment:
       Returning the original batch will be faster.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }

Review comment:
       Also, if `values.null_count()` is 0, you can probably return the array unchanged.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {

Review comment:
       This kind of checks is faster as `values->type()->id() == Type::ID`.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), global_offset);
+      }
+      global_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(table.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(table), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.table();

Review comment:
       You could return a `Datum` from all these `drop_null` implementations, instead of converting Datum to table then Datum.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {

Review comment:
       Why?
   1) The filter array created by `GetDropNullFilter` doesn't have any nulls.
   2) It's better to check this _before_ creating the filter, i.e. `if (values->null_count() == values->length()) ...`

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);

Review comment:
       Nit: `std::move(empty_batch)`

##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def drop_null(data, *, memory_pool=None):

Review comment:
       This shouldn't be necessary unless you want to craft a special function signature (which doesn't seem to be the case here).

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());

Review comment:
       Better to return the original table, here and above.

##########
File path: python/pyarrow/table.pxi
##########
@@ -397,6 +397,13 @@ cdef class ChunkedArray(_PandasConvertible):
         """
         return _pc().take(self, indices)
 
+    def drop_null(self):
+        """
+        Remove missing values from a chunked array.
+        See pyarrow.compute.drop_null for full usage.

Review comment:
       "for full description" perhaps, because the function signature (hence usage) is trivial?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);

Review comment:
       Hmm... is this necessary? `BitmapAnd` allows you to pass arbitrary input offsets, so you shouldn't need to concatenate all source bitmap chunks for the column.

##########
File path: python/pyarrow/table.pxi
##########
@@ -956,6 +963,13 @@ cdef class RecordBatch(_PandasConvertible):
         """
         return _pc().take(self, indices)
 
+    def drop_null(self):
+        """
+        Remove missing values from an RecordBatch.

Review comment:
       I would say "a record batch" or "a RecordBatch".

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,551 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");

Review comment:
       Ideally, should also check with an empty array and a nulls-only array.

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,82 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_drop_null(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.drop_null()
+    result.validate()

Review comment:
       `result.validate(full=True)` is more thorough.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,551 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  template <typename OffsetType>
+  std::vector<OffsetType> Offsets(int32_t length, int32_t slice_count) {
+    std::vector<OffsetType> offsets(static_cast<std::size_t>(slice_count + 1));
+    std::default_random_engine gen(seed_);
+    std::uniform_int_distribution<OffsetType> dist(0, length);
+    std::generate(offsets.begin(), offsets.end(), [&] { return dist(gen); });
+    std::sort(offsets.begin(), offsets.end());
+    return offsets;
+  }

Review comment:
       Can probably reuse `RandomArrayGenerator::Offsets` instead.

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,82 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_drop_null(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.drop_null()
+    result.validate()
+    indices = [i for i in range(len(arr)) if arr[i].is_valid]
+    expected = arr.take(pa.array(indices))
+    assert result.equals(expected)
+
+
+def test_drop_null_chunked_array():
+    arr = pa.chunked_array([["a", None], ["c", "d", None], [None], []])
+    expected_drop = pa.chunked_array([["a"], ["c", "d"], [], []])
+
+    result = arr.drop_null()
+    assert result.equals(expected_drop)
+
+
+def test_drop_null_record_batch():
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None])], names=["a'"])
+    result = batch.drop_null()
+    expected = pa.record_batch([pa.array(["a", "c", "d"])], names=["a'"])
+    assert result.equals(expected)
+
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None]),
+         pa.array([None, None, "c", None, "e"])], names=["a'", "b'"])
+
+    result = batch.drop_null()
+    expected = pa.record_batch(
+        [pa.array(["c"]), pa.array(["c"])], names=["a'", "b'"])
+    assert result.equals(expected)
+
+
+def test_drop_null_table():
+    table = pa.table([pa.array(["a", None, "c", "d", None])], names=["a"])
+    expected = pa.table([pa.array(["a", "c", "d"])], names=["a"])
+    result = table.drop_null()
+    assert result.equals(expected)
+
+    table = pa.table([pa.chunked_array([["a", None], ["c", "d", None]]),
+                      pa.chunked_array([["a", None], [None, "d", None]]),
+                      pa.chunked_array([["a"], ["b"], [None], ["d", None]])],
+                     names=["a", "b", "c"])
+    expected = pa.table([pa.array(["a", "d"]),
+                         pa.array(["a", "d"]),
+                         pa.array(["a", "d"])],
+                        names=["a", "b", "c"])
+    result = table.drop_null()
+    assert result.equals(expected)
+
+    table = pa.table([pa.chunked_array([["a", "b"], ["c", "d", "e"]]),
+                      pa.chunked_array([["a"], ["b"], [None], ["d", None]]),
+                      pa.chunked_array([["a", None], ["c", "d", None]])],
+                     names=["a", "b", "c"])
+    expected = pa.table([pa.array(["a", "d"]),
+                         pa.array(["a", "d"]),
+                         pa.array(["a", "d"])],
+                        names=["a", "b", "c"])

Review comment:
       Hmm, you could probably make the values different in each column, though this is already tested on the C++ side, so probably not very important.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);

Review comment:
       Another possibility entirely is to get `TableBatchReader` to get a vector of record batches spanning the table, then call drop_null on each record batch and then `Table::FromRecordBatches` to get the resulting table.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   It's in https://issues.apache.org/jira/browse/ARROW-13665


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa edited a comment on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   I updated the PR addressing feedback. 
   
   The following test cases were added.
   - 0-lengthed inputs (to test the early termination code paths)
   - non-zero but all null values on inputs (to test the early termination code paths)
   - Larger test case with more random values (say 1000 or so) --> Because bitmap ops work on 8 elements together. So, we need to test beyond 8 elements in an array.
   - Test with non-zero offsets in the inputs (for chunkedarrays and tables)
   
   cc @nirandaperera, @westonpace 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   > One last comment: can you document the function in `docs/source/cpp/compute.rst`?
   
   Thanks for the feedback @pitrou. 
   I updated this PR :) 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -591,6 +591,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def drop_null(data, memory_pool=None):

Review comment:
       I'll remove this as the proper definition is auto-generated from the C++ metadata.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,548 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+  CheckDropNull(boolean(), "[]", "[]");
+  CheckDropNull(boolean(), "[null, null]", "[]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  std::shared_ptr<Int32Array> Offsets(int32_t length, int32_t slice_count) {
+    return checked_pointer_cast<Int32Array>(rng_.Offsets(slice_count, 0, length));
+  }
+
+  ArrayVector Slices(const std::shared_ptr<Array>& array,
+                     const std::shared_ptr<Int32Array>& offsets) {
+    ArrayVector slices(offsets->length() - 1);
+    for (int64_t i = 0; i != static_cast<int64_t>(slices.size()); ++i) {
+      slices[i] =
+          array->Slice(offsets->Value(i), offsets->Value(i + 1) - offsets->Value(i));
+    }
+    return slices;
+  }
+
+  random::SeedType seed_ = 0xdeadbeef;
+  random::RandomArrayGenerator rng_;
+};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryArrowTypes);
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullString) {
+  this->AssertDropNull(R"(["a", "b", "c"])", R"(["a", "b", "c"])");
+  this->AssertDropNull(R"([null, "b", "c"])", "[\"b\", \"c\"]");
+  this->AssertDropNull(R"(["a", "b", null])", R"(["a", "b"])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(), R"(["a", "b", "c"])");
+}
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullDictionary) {
+  auto dict = R"(["a", "b", "c", "d", "e"])";
+  this->AssertDropNullDictionary(dict, "[3, 4, 2]", "[3, 4, 2]");
+  this->AssertDropNullDictionary(dict, "[null, 4, 2]", "[4, 2]");
+}
+
+class TestDropNullKernelFSB : public TestDropNullKernelTyped<FixedSizeBinaryType> {
+ public:
+  std::shared_ptr<DataType> value_type() { return fixed_size_binary(3); }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+};
+
+TEST_F(TestDropNullKernelFSB, DropNullFixedSizeBinary) {
+  this->AssertDropNull(R"(["aaa", "bbb", "ccc"])", R"(["aaa", "bbb", "ccc"])");
+  this->AssertDropNull(R"([null, "bbb", "ccc"])", "[\"bbb\", \"ccc\"]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(),
+                                                R"(["aaa", "bbb", "ccc"])");
+}
+
+class TestDropNullKernelWithList : public TestDropNullKernelTyped<ListType> {};
+
+TEST_F(TestDropNullKernelWithList, DropNullListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(list(int32()), list_json, "[[], [1,2], [3]]");
+  this->TestNoValidityBitmapButUnknownNullCount(list(int32()), "[[], [1,2], [3]]");
+}
+
+TEST_F(TestDropNullKernelWithList, DropNullListListInt32) {
+  std::string list_json = R"([
+    [],
+    [[1], [2, null, 2], []],
+    null,
+    [[3, null], null]
+  ])";
+  auto type = list(list(int32()));
+  CheckDropNull(type, list_json, R"([
+    [],
+    [[1], [2, null, 2], []],
+    [[3, null], null]
+  ])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(type,
+                                                "[[[1], [2, null, 2], []], [[3, null]]]");
+}
+
+class TestDropNullKernelWithLargeList : public TestDropNullKernelTyped<LargeListType> {};
+
+TEST_F(TestDropNullKernelWithLargeList, DropNullLargeListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(large_list(int32()), list_json, "[[], [1,2],  [3]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithFixedSizeList
+    : public TestDropNullKernelTyped<FixedSizeListType> {};
+
+TEST_F(TestDropNullKernelWithFixedSizeList, DropNullFixedSizeListInt32) {
+  std::string list_json = "[null, [1, null, 3], [4, 5, 6], [7, 8, null]]";
+  CheckDropNull(fixed_size_list(int32(), 3), list_json,
+                "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithMap : public TestDropNullKernelTyped<MapType> {};
+
+TEST_F(TestDropNullKernelWithMap, DropNullMapStringToInt32) {
+  std::string map_json = R"([
+    [["joe", 0], ["mark", null]],
+    null,
+    [["cap", 8]],
+    []
+  ])";
+  std::string expected_json = R"([
+    [["joe", 0], ["mark", null]],
+    [["cap", 8]],
+    []
+  ])";
+  CheckDropNull(map(utf8(), int32()), map_json, expected_json);
+}
+
+class TestDropNullKernelWithStruct : public TestDropNullKernelTyped<StructType> {};
+
+TEST_F(TestDropNullKernelWithStruct, DropNullStruct) {
+  auto struct_type = struct_({field("a", int32()), field("b", utf8())});
+  auto struct_json = R"([
+    null,
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  auto expected_struct_json = R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  CheckDropNull(struct_type, struct_json, expected_struct_json);
+  this->TestNoValidityBitmapButUnknownNullCount(struct_type, expected_struct_json);
+}
+
+class TestDropNullKernelWithUnion : public TestDropNullKernelTyped<UnionType> {};
+
+TEST_F(TestDropNullKernelWithUnion, DropNullUnion) {
+  auto union_type = dense_union({field("a", int32()), field("b", utf8())}, {2, 5});
+  auto union_json = R"([
+      [2, null],
+      [2, 222],
+      [5, "hello"],
+      [5, "eh"],
+      [2, null],
+      [2, 111],
+      [5, null]
+    ])";
+  CheckDropNull(union_type, union_json, union_json);
+}
+
+class TestDropNullKernelWithRecordBatch : public TestDropNullKernelTyped<RecordBatch> {
+ public:
+  void AssertDropNull(const std::shared_ptr<Schema>& schm, const std::string& batch_json,
+                      const std::string& expected_batch) {
+    std::shared_ptr<RecordBatch> actual;
+
+    ASSERT_OK(this->DoDropNull(schm, batch_json, &actual));
+    ValidateOutput(actual);
+    ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(schm, expected_batch), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<Schema>& schm, const std::string& batch_json,
+                    std::shared_ptr<RecordBatch>* out) {
+    auto batch = RecordBatchFromJSON(schm, batch_json);
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(batch));
+    *out = out_datum.record_batch();
+    return Status::OK();
+  }
+};
+
+TEST_F(TestDropNullKernelWithRecordBatch, DropNullRecordBatch) {
+  std::vector<std::shared_ptr<Field>> fields = {field("a", int32()), field("b", utf8())};
+  auto schm = schema(fields);
+
+  auto batch_json = R"([
+    {"a": null, "b": "yo"},
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  this->AssertDropNull(schm, batch_json, R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])");
+
+  batch_json = R"([
+    {"a": null, "b": "yo"},
+    {"a": 1, "b": null},
+    {"a": null, "b": "hello"},
+    {"a": 4, "b": null}
+  ])";
+  this->AssertDropNull(schm, batch_json, R"([])");
+  this->AssertDropNull(schm, R"([])", R"([])");
+}
+
+class TestDropNullKernelWithChunkedArray : public TestDropNullKernelTyped<ChunkedArray> {
+ public:
+  TestDropNullKernelWithChunkedArray()
+      : sizes_({0, 1, 2, 4, 16, 31, 1234}),
+        null_probabilities_({0.0, 0.1, 0.5, 0.9, 1.0}) {}
+
+  void AssertDropNull(const std::shared_ptr<DataType>& type,
+                      const std::vector<std::string>& values,
+                      const std::vector<std::string>& expected) {
+    std::shared_ptr<ChunkedArray> actual;
+    ASSERT_OK(this->DoDropNull(type, values, &actual));
+    ValidateOutput(actual);
+
+    AssertChunkedEqual(*ChunkedArrayFromJSON(type, expected), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<DataType>& type,
+                    const std::vector<std::string>& values,
+                    std::shared_ptr<ChunkedArray>* out) {
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(ChunkedArrayFromJSON(type, values)));
+    *out = out_datum.chunked_array();
+    return Status::OK();
+  }
+
+  template <typename ArrayFactory>
+  void CheckDropNullWithSlices(ArrayFactory&& factory) {
+    for (auto size : this->sizes_) {
+      for (auto null_probability : this->null_probabilities_) {
+        std::shared_ptr<Array> concatenated_array;
+        std::shared_ptr<ChunkedArray> chunked_array;
+        factory(size, null_probability, &chunked_array, &concatenated_array);
+
+        ASSERT_OK_AND_ASSIGN(auto out_datum, DropNull(chunked_array));
+        auto actual_chunked_array = out_datum.chunked_array();
+        ASSERT_OK_AND_ASSIGN(auto actual, Concatenate(actual_chunked_array->chunks()));
+
+        ASSERT_OK_AND_ASSIGN(out_datum, DropNull(*concatenated_array));
+        auto expected = out_datum.make_array();
+
+        AssertArraysEqual(*expected, *actual);
+      }
+    }
+  }
+
+  std::vector<int32_t> sizes_;
+  std::vector<double> null_probabilities_;
+};
+
+TEST_F(TestDropNullKernelWithChunkedArray, DropNullChunkedArray) {
+  this->AssertDropNull(int8(), {"[]"}, {"[]"});
+  this->AssertDropNull(int8(), {"[null]", "[8, null]"}, {"[8]"});
+
+  this->AssertDropNull(int8(), {"[null]", "[null, null]"}, {"[]"});
+  this->AssertDropNull(int8(), {"[7]", "[8, 9]"}, {"[7]", "[8, 9]"});
+  this->AssertDropNull(int8(), {"[]", "[]"}, {"[]", "[]"});
+}
+
+TEST_F(TestDropNullKernelWithChunkedArray, DropNullChunkedArrayWithSlices) {
+  // With Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<ChunkedArray>* out_chunked_array,
+                                       std::shared_ptr<Array>* out_concatenated_array) {
+    auto array = std::make_shared<NullArray>(size);
+    auto offsets = this->Offsets(size, 3);
+    auto slices = this->Slices(array, offsets);
+    *out_chunked_array = std::make_shared<ChunkedArray>(std::move(slices));
+
+    ASSERT_OK_AND_ASSIGN(*out_concatenated_array,
+                         Concatenate((*out_chunked_array)->chunks()));
+  });
+  // Without Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<ChunkedArray>* out_chunked_array,
+                                       std::shared_ptr<Array>* out_concatenated_array) {
+    auto array = this->rng_.ArrayOf(int16(), size, null_probability);
+    auto offsets = this->Offsets(size, 3);
+    auto slices = this->Slices(array, offsets);
+    *out_chunked_array = std::make_shared<ChunkedArray>(std::move(slices));
+
+    ASSERT_OK_AND_ASSIGN(*out_concatenated_array,
+                         Concatenate((*out_chunked_array)->chunks()));
+  });
+}
+
+class TestDropNullKernelWithTable : public TestDropNullKernelTyped<Table> {
+ public:
+  TestDropNullKernelWithTable()
+      : sizes_({0, 1, 2, 4, 16, 31, 1234}),
+        null_probabilities_({0.0, 0.1, 0.5, 0.9, 1.0}) {}
+
+  void AssertDropNull(const std::shared_ptr<Schema>& schm,
+                      const std::vector<std::string>& table_json,
+                      const std::vector<std::string>& expected_table) {
+    std::shared_ptr<Table> actual;
+    ASSERT_OK(this->DoDropNull(schm, table_json, &actual));
+    ValidateOutput(actual);
+    ASSERT_TABLES_EQUAL(*TableFromJSON(schm, expected_table), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<Schema>& schm,
+                    const std::vector<std::string>& values, std::shared_ptr<Table>* out) {
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(TableFromJSON(schm, values)));
+    *out = out_datum.table();
+    return Status::OK();
+  }
+
+  template <typename ArrayFactory>
+  void CheckDropNullWithSlices(ArrayFactory&& factory) {
+    for (auto size : this->sizes_) {
+      for (auto null_probability : this->null_probabilities_) {
+        std::shared_ptr<Table> table_w_slices;
+        std::shared_ptr<Table> table_wo_slices;
+
+        factory(size, null_probability, &table_w_slices, &table_wo_slices);
+
+        ASSERT_OK_AND_ASSIGN(auto out_datum, DropNull(table_w_slices));
+        auto actual = out_datum.table();
+
+        ASSERT_OK_AND_ASSIGN(out_datum, DropNull(table_wo_slices));
+        auto expected = out_datum.table();
+        if (actual->num_rows() > 0) {
+          ASSERT_TRUE(actual->num_rows() == expected->num_rows());
+          for (int index = 0; index < actual->num_columns(); index++) {
+            ASSERT_OK_AND_ASSIGN(auto actual_col,
+                                 Concatenate(actual->column(index)->chunks()));
+            ASSERT_OK_AND_ASSIGN(auto expected_col,
+                                 Concatenate(expected->column(index)->chunks()));
+            AssertArraysEqual(*actual_col, *expected_col);
+          }
+        }
+      }
+    }
+  }
+
+  std::vector<int32_t> sizes_;
+  std::vector<double> null_probabilities_;
+};
+
+TEST_F(TestDropNullKernelWithTable, DropNullTable) {
+  std::vector<std::shared_ptr<Field>> fields = {field("a", int32()), field("b", utf8())};
+  auto schm = schema(fields);
+
+  {
+    std::vector<std::string> table_json = {R"([
+      {"a": null, "b": "yo"},
+      {"a": 1, "b": ""}
+    ])",
+                                           R"([
+      {"a": 2, "b": "hello"},
+      {"a": 4, "b": "eh"}
+    ])"};
+    std::vector<std::string> expected_table_json = {R"([
+      {"a": 1, "b": ""}
+    ])",
+                                                    R"([
+      {"a": 2, "b": "hello"},
+      {"a": 4, "b": "eh"}
+    ])"};
+    this->AssertDropNull(schm, table_json, expected_table_json);
+  }
+  {
+    std::vector<std::string> table_json = {R"([
+      {"a": null, "b": "yo"},
+      {"a": 1, "b": null}
+    ])",
+                                           R"([
+      {"a": 2, "b": null},
+      {"a": null, "b": "eh"}
+    ])"};
+    std::shared_ptr<Table> actual;
+    ASSERT_OK(this->DoDropNull(schm, table_json, &actual));
+    ASSERT_EQ(actual->num_rows(), 0);
+  }
+}
+
+TEST_F(TestDropNullKernelWithTable, DropNullTableWithWithSlices) {
+  // With Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<Table>* out_table_w_slices,
+                                       std::shared_ptr<Table>* out_table_wo_slices) {
+    std::vector<std::shared_ptr<Field>> fields = {field("a", int32()),
+                                                  field("b", utf8())};
+    auto schm = schema(fields);
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(fields.size());
+    auto col_a = std::make_shared<NullArray>(size);
+    auto col_b = std::make_shared<NullArray>(size);

Review comment:
       These are the wrong column types. I will fix and add a validation.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());

Review comment:
       Same comment again as for the record batch variant.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);

Review comment:
       Ping on this?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def dropnull(data, *, memory_pool=None):

Review comment:
       The explicit functions are mostly historical (auto-generation became less and less limited over time), but in some cases they allow for more user-friendly signatures. If your explicit function has the same signature as the auto-generated one, then it's basically 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,

Review comment:
       Hmm, I'm curious why @nirandaperera suggested this. Returning `Result<...>` is the recommended convention in all newly written code (an exception is when the out-parameter is optional, and/or you have several of them).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t length_count = 0;
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  for (int i = 0; i < batch.num_columns(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(auto indices,
+                          GetNotNullIndices(batch.column(i), ctx->memory_pool()));
+    ARROW_ASSIGN_OR_RAISE(Datum out, Take(batch.column(i)->data(), Datum(indices),
+                                          TakeOptions::NoBoundsCheck(), ctx));
+    columns[i] = out.make_array();
+    length_count += columns[i]->length();
+  }
+  return RecordBatch::Make(batch.schema(), length_count, std::move(columns));
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {

Review comment:
       Sure makes 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,551 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  template <typename OffsetType>
+  std::vector<OffsetType> Offsets(int32_t length, int32_t slice_count) {
+    std::vector<OffsetType> offsets(static_cast<std::size_t>(slice_count + 1));
+    std::default_random_engine gen(seed_);
+    std::uniform_int_distribution<OffsetType> dist(0, length);
+    std::generate(offsets.begin(), offsets.end(), [&] { return dist(gen); });
+    std::sort(offsets.begin(), offsets.end());
+    return offsets;
+  }

Review comment:
       I tried, but the specific number of offsets doesn't match all the time with null_probability  input for RandomArrayGenerator::Offsets generator. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] commented on pull request #10802: [C++] Implement "drop null" kernels that return array without nulls

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10802:
URL: https://github.com/apache/arrow/pull/10802#issuecomment-886803563


   <!--
     Licensed to the Apache Software Foundation (ASF) under one
     or more contributor license agreements.  See the NOTICE file
     distributed with this work for additional information
     regarding copyright ownership.  The ASF licenses this file
     to you under the Apache License, Version 2.0 (the
     "License"); you may not use this file except in compliance
     with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
     Unless required by applicable law or agreed to in writing,
     software distributed under the License is distributed on an
     "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
     KIND, either express or implied.  See the License for the
     specific language governing permissions and limitations
     under the License.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def dropnull(data, *, memory_pool=None):

Review comment:
       Is this explicit function needed? (the auto-generated function doesn't work, or are the vector functions not auto-generated?)

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,185 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> dropnull_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &dropnull_filter));
+
+  if (dropnull_filter->null_count() == dropnull_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*dropnull_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  std::set<int32_t> notnull_indices;
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    for (int64_t i = 0; i < column->length(); ++i) {
+      if (!column->IsValid(i)) {
+        notnull_indices.insert(static_cast<int32_t>(i));
+      }
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(batch.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < batch.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeRA(batch, *indices, TakeOptions::Defaults(), ctx);
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(static_cast<int32_t>(relative_index + i));
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+}
+
+const FunctionDoc dropnull_doc(
+    "DropNull kernel",
+    ("The output is populated with values from the input without the null values"),

Review comment:
       Given that this also supports RecordBatches/Tables, I think it would be good to explicitly describe the behaviour in those cases (as there are multiple options, eg the any/all in pandas).

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,185 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> dropnull_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &dropnull_filter));
+
+  if (dropnull_filter->null_count() == dropnull_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*dropnull_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  std::set<int32_t> notnull_indices;
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    for (int64_t i = 0; i < column->length(); ++i) {
+      if (!column->IsValid(i)) {
+        notnull_indices.insert(static_cast<int32_t>(i));
+      }
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(batch.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < batch.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeRA(batch, *indices, TakeOptions::Defaults(), ctx);
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(static_cast<int32_t>(relative_index + i));
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+}
+
+const FunctionDoc dropnull_doc(
+    "DropNull kernel",

Review comment:
       ```suggestion
       "Drop nulls from the input",
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,185 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> dropnull_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &dropnull_filter));
+
+  if (dropnull_filter->null_count() == dropnull_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*dropnull_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  std::set<int32_t> notnull_indices;
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    for (int64_t i = 0; i < column->length(); ++i) {
+      if (!column->IsValid(i)) {
+        notnull_indices.insert(static_cast<int32_t>(i));
+      }
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(batch.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < batch.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeRA(batch, *indices, TakeOptions::Defaults(), ctx);
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(static_cast<int32_t>(relative_index + i));
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  if (static_cast<int64_t>(notnull_indices.size()) == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  RETURN_NOT_OK(
+      builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size())));
+  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
+    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+}
+
+const FunctionDoc dropnull_doc(
+    "DropNull kernel",
+    ("The output is populated with values from the input without the null values"),
+    {"input"});
+
+class DropNullMetaFunction : public MetaFunction {
+ public:
+  DropNullMetaFunction() : MetaFunction("dropnull", Arity::Unary(), &dropnull_doc) {}

Review comment:
       ```suggestion
     DropNullMetaFunction() : MetaFunction("drop_null", Arity::Unary(), &dropnull_doc) {}
   ```
   
   (I think we generally use underscores, eg we already have `"is_null"` kernel)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       I'm a little confused on the sematics for `DropNull` in record batches. What would happen in the following case?
   
   | A | B    | C    |
   |---|------|------|
   | 1 | 4    | null |
   | 2 | null | 6    |
   | 3 | 5    | null |
   

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {

Review comment:
       I feel like it would be nice to have the following signature
   ```c++
   Status GetNotNullIndices( const std::shared_ptr<Array>& column, MemoryPool* memory_pool, std::shared_ptr<arrow::Array>* out_array);
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       I have a feeling that we need to make a union out of all the validity buffers and then call `Take` based on that?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t length_count = 0;
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  for (int i = 0; i < batch.num_columns(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(auto indices,
+                          GetNotNullIndices(batch.column(i), ctx->memory_pool()));
+    ARROW_ASSIGN_OR_RAISE(Datum out, Take(batch.column(i)->data(), Datum(indices),
+                                          TakeOptions::NoBoundsCheck(), ctx));
+    columns[i] = out.make_array();
+    length_count += columns[i]->length();
+  }
+  return RecordBatch::Make(batch.schema(), length_count, std::move(columns));
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(relative_index + i);
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size()));
+  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
+    if (notnull_indices.find(row_index) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+}
+
+const FunctionDoc dropnull_doc(
+    "DropNull kernel",
+    ("The output is populated with values from the input without the null values"),
+    {"input"});
+
+class DropNullMetaFunction : public MetaFunction {

Review comment:
       There's an alternative approach to using `Take` (at least for arrays). We can promote array's validity buffer to a BooleanArray and call `Filter`. In that case, we dont have to create the valid indices IMO

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1718,5 +1718,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {

Review comment:
       nit: you can keep this as a class member




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);

Review comment:
       I followed your last suggest using `TableBatchReader`. The reason that I needed to concatenate all source bitmap chunks was that `BitmapAnd` only has an unaligned implementation with respect to the offset (I don't why) not the bitmap length. So if `chunk->length() % 8 != 0` `BitmapAnd` will fail. https://github.com/apache/arrow/blob/820e5061847c9d6d261c416e57d6013321175565/cpp/src/arrow/util/bitmap_ops.cc#L285
   
   Referenced before
   https://github.com/apache/arrow/pull/10802#discussion_r684341804




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());

Review comment:
       Every function which returns Status or Result<> will prompt a similar compiler error if not explicitly handled, since it could be ignoring (for example) an OOM Status https://github.com/apache/arrow/pull/10802/checks?check_run_id=3202879387#step:5:608
   ```suggestion
     RETURN_NOT_OK(builder.Reserve(column->length() - column->null_count()));
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   I updated this PR addressing latest feedback comments.
   I think this PR is ready for merging. Let me know if anything needs to be taken care of from my end. cc @nirandaperera, @jorisvandenbossche, @westonpace 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   Lol! Yes. I think that was a mistake from me! 😭
   
   On Fri, Aug 13, 2021, 11:05 Antoine Pitrou ***@***.***> wrote:
   
   > ***@***.**** commented on this pull request.
   > ------------------------------
   >
   > In cpp/src/arrow/compute/kernels/vector_selection.cc
   > <https://github.com/apache/arrow/pull/10802#discussion_r688584422>:
   >
   > > @@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   >    }
   >  };
   >
   > +// ----------------------------------------------------------------------
   > +// DropNull Implementation
   > +
   > +Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
   >
   > Hmm, I'm curious why @nirandaperera <https://github.com/nirandaperera>
   > suggested this. Returning Result<...> is the recommended convention in
   > all newly written code (an exception is when the out-parameter is optional,
   > and/or you have several of them).
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/arrow/pull/10802#discussion_r688584422>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/ABKS65PYEWOTPW62DJEI2KDT4UYCXANCNFSM5BAK5SZA>
   > .
   >
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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






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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   > Note the auto-generated Python docstring isn't entirely accurate (see the `input` argument description):
   > 
   > ```
   > Signature: pc.drop_null(input, *, memory_pool=None)
   > Docstring:
   > Drop nulls from the input.
   > 
   > The output is populated with values from the input (Array, ChunkedArray,
   > RecordBatch, or Table) without the null values.
   > For the RecordBatch and Table cases, `drop_null` drops the full row if
   > there is any null.
   > 
   > Parameters
   > ----------
   > input : Array-like or scalar-like
   >     Argument to compute function
   > memory_pool : pyarrow.MemoryPool, optional
   >     If not passed, will allocate memory from the default memory pool.
   > File:      ~/arrow/dev/python/pyarrow/compute.py
   > Type:      function
   > ```
   > 
   > I'll open a separate JIRA.
   
   Ok, thanks @pitrou let me know in which jira ticket will be this  to follow up the updates. 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }

Review comment:
       nit:
   ```suggestion
     for (const auto& column: batch.columns()) {
       null_count += column->null_count();
     }
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }

Review comment:
       nit: range loops are much cleaner, and aesthetically pleasing :wink: 
   ```suggestion
     for (const auto& col: table.columns()) {
       for (const auto& column_chunk: col->chunks()) {
         null_count += column_chunk->null_count();
       }
     }
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }

Review comment:
       same comment as in the record batch

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }
+      bitmap_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }

Review comment:
       this can be simplified IMO
   
   ```suggestion
      ARROW_ASSIGN_OR_RAISE(auto dst,
                           AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
     BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
   
     for (const auto& col: table.columns()) {
         int64_t global_offset = 0;
         for (const auto& chunk: col.chunks()) {
           if (chunk-> null_bitmap()){
             // BitmapAnd only on the corresponding region of dst
           ::arrow::internal::BitmapAnd(dst->data(), global_offset, chunk->data(), chunk->offset(),
                                    chunk->length(), global_offset, dst->mutable_data());
           }
           global_offset += chunk->length();
         }
     }
   ```

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }

Review comment:
       I wouldn't this TBH. This doesn't cover all cases where we could return an empty table/ record batch. 
   
   I think the best would be to defer it until we have the `dst` array populated. 
   
   if `dst->null_count() == def->length()` we could return an empty Table/ RecordBatch. And that would cover all cases where we need to return an empty table. :-) 

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1734,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------

Review comment:
       I think we need to add following test cases
   1. 0-lengthed inputs  (to test the early termination code paths)
   2. non-zero but all null values on inputs  (to test the early termination code paths)
   3. larger test case with more random values (say 1000 or so) --> Because bitmap ops work on 8 elements together.  So, we need to test beyond 8 elements in an array. 
   ex: https://github.com/apache/arrow/blob/master/cpp/src/arrow/compute/kernels/scalar_if_else_test.cc#L56
   4. also, it would be nicer to have test with non-zero offsets in the inputs (only for array impl)




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   > LGTM. +1 @aocsa Thank you for doing this! :-)
   
   cc @westonpace 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }
+      bitmap_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }

Review comment:
       This change can't be done. `BitmapAnd` only has an `unaligned`  implemnentation with respect to the offset (I don't why). So if `chunk->length() % 8 != 0` BitmapAnd will fail. One way to do this is to create an aligned and concatenated bitmap for each column and them apply the bytewise BitmapAnd.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   Thanks @nirandaperera. I updated the PR based on latest feedback. Nice to know more about the Arrow BitUtils :)
   IMO this PR is ready to be merged :) +1 cc @westonpace 
   
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }

Review comment:
       yeap, https://github.com/apache/arrow/blob/2675cb94ca26126f21a93ed596347545f1e72928/cpp/src/arrow/compute/kernels/vector_selection.cc#L2181 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }

Review comment:
       Sure, my mistake




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2146,184 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    } else {
+      for (int64_t i = 0; i < column->length(); ++i) {
+        if (!column->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), i);
+        }
+      }
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+
+  if (drop_null_filter->false_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+  // Note: Not all chunks has null_bitmap data, so we are using IsValid method
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < static_cast<int64_t>(inputs[col].size());
+         ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), relative_index + i);

Review comment:
       Thanks @nirandaperera. I updated the PR based on latest feedback. Nice to know more about the Arrow BitUtils :) 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       I see... 
   so then another approach would be to do a `internal::CopyBitmap` and inplace `internal::BitmapAnd` on all validity buffers (non-nullptr) and do a selection. But I am not sure if that would be faster than the `std::set` approach. My gut feeling is it would be, because we dont need to do any searches on the `std::set` (and inserts, both of which are log N) before creating the take array, and std::set nodes are allocated dynamically (which could have cache inefficiencies). Bitmap approach works on contiguous memory and ops would get vectorized. 
   
   Ref:
   https://github.com/apache/arrow/blob/master/cpp/src/arrow/util/bitmap_ops.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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}

Review comment:
       https://issues.apache.org/jira/browse/ARROW-13579




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   I updated the PR addressing feedback. 
   
   The following test cases were added.
   - 0-lengthed inputs (to test the early termination code paths)
   - non-zero but all null values on inputs (to test the early termination code paths)
   - Larger test case with more random values (say 1000 or so) --> Because bitmap ops work on 8 elements together. So, we need to test beyond 8 elements in an array.
   - Test with non-zero offsets in the inputs (for chunkedarrays and tables)
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());

Review comment:
       Ping here. If you pass this function a `const std::shared_ptr<RecordBatch>& batch` argument, you can just `return batch` here AFAICT.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }

Review comment:
       CopyBitmap can't be skipped for now.  BitmapAnd only has an unaligned implemnentation with respect to the offset (I don't why). So if chunk->length() % 8 != 0 BitmapAnd will fail. I would suggest to create a follow-up issue for this. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());

Review comment:
       Still worth addressing.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), global_offset);
+      }
+      global_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(table.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(table), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.table();
+}
+
+const FunctionDoc drop_null_doc(
+    "Drop nulls from the input",
+    ("The output is populated with values from the input (Array, ChunkedArray, "
+     "RecordBatch, or Table) without the null values."
+     "Note that for the RecordBatch/Table cases, `drop_null` drops the full row if there "
+     "is any null."),

Review comment:
       Ah, one other thing: as a convention, we wrap FunctionDoc descriptions with explicit newlines (`\n`). It would be good to conform to that here.
   




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def dropnull(data, *, memory_pool=None):

Review comment:
       To the best of my knowledge, this function is needed to be exposed as part of `pyarrow.compute` module https://github.com/apache/arrow/blob/082b11e9ecb4acd60d2e36ba8b5b3bda8b2027ce/python/pyarrow/lib.pyx#L113
    




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2146,184 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {

Review comment:
       Shall we have an early termination here, that checks if none of the columns have nulls, simply return the input?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2146,184 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    } else {
+      for (int64_t i = 0; i < column->length(); ++i) {
+        if (!column->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), i);
+        }
+      }
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+
+  if (drop_null_filter->false_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+

Review comment:
       we can have an early termination for  tables as well IMO (if all chunkedarrays are non-null, just return input)

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2146,184 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    } else {
+      for (int64_t i = 0; i < column->length(); ++i) {
+        if (!column->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), i);
+        }
+      }
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+
+  if (drop_null_filter->false_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+  // Note: Not all chunks has null_bitmap data, so we are using IsValid method
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < static_cast<int64_t>(inputs[col].size());
+         ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), relative_index + i);

Review comment:
       I think you can use the `BitmapAnd` approach you used in record batches here. Only thing is you'll have to manipulate the offsets and lengths properly :slightly_smiling_face: 

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2146,184 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    } else {
+      for (int64_t i = 0; i < column->length(); ++i) {
+        if (!column->IsValid(i)) {
+          BitUtil::ClearBit(dst->mutable_data(), i);
+        }

Review comment:
       I think this else block is redundant. `column->null_bitmap_data() == nullptr` means that all are valid. So we can omit this branch!




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,216 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), global_offset);
+      }
+      global_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(table.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(table), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.table();
+}
+
+const FunctionDoc drop_null_doc(
+    "Drop Null kernel",
+    ("The output is populated with values from the input (Array, ChunkedArray, "
+     "RecordBatch, or Table) without the null values"),

Review comment:
       I think it would be good to explicitly describe the behaviour for the RecordBatch/Table cases, i.e. that it drops the full row if there is any null (as there are multiple options, it could also have been "all" instead of "any").

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2273,27 +2311,20 @@ Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ct
     }
     return Table::Make(table.schema(), empty_table, 0);
   }
-  std::shared_ptr<arrow::Array> indices;
-  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
-  RETURN_NOT_OK(
-      builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size())));
-  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
-    if (notnull_indices.find(static_cast<int32_t>(row_index)) == notnull_indices.end()) {
-      builder.UnsafeAppend(static_cast<int32_t>(row_index));
-    }
-  }
-  RETURN_NOT_OK(builder.Finish(&indices));
-  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(table), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.table();
 }
 
-const FunctionDoc dropnull_doc(
-    "DropNull kernel",
-    ("The output is populated with values from the input without the null values"),
+const FunctionDoc drop_null_doc(
+    "Drop Null kernel",

Review comment:
       ```suggestion
       "Drop nulls from the input",
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       Yes, the output would be an empty table. I am following the pandas dropna API
   https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.dropna.html




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou closed pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

Posted by GitBox <gi...@apache.org>.
pitrou closed pull request #10802:
URL: https://github.com/apache/arrow/pull/10802


   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       DropNullRecordBatch now uses internally GetTakeIndicesImpl and it takes into consideration the offset.  




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   > @aocsa with your latest push, it seems you (accidentally?) undid some previous changes, eg the name "drop_null" went back the "dropnull" (not sure if there are others).
   > (without adding new commits with your latest changes (but amending them to previous ones), it's difficult to see what actually changed)
   > 
   > Also, you resolved my two small doc comments of yesterday, but it doesn't seem something actually changed on those lines?
   
   Yeap It was an accident.  I change back the change. Thanks for notice this. cc @jorisvandenbossche 
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,551 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  template <typename OffsetType>
+  std::vector<OffsetType> Offsets(int32_t length, int32_t slice_count) {
+    std::vector<OffsetType> offsets(static_cast<std::size_t>(slice_count + 1));
+    std::default_random_engine gen(seed_);
+    std::uniform_int_distribution<OffsetType> dist(0, length);
+    std::generate(offsets.begin(), offsets.end(), [&] { return dist(gen); });
+    std::sort(offsets.begin(), offsets.end());
+    return offsets;
+  }

Review comment:
       Hmm, I'm not sure I understand what you mean here. You should just pass a zero null_probability for offsets (the default).




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }

Review comment:
       Right, but I don't see the same line here in `DropNullChunkedArray` :-) Am I misunderstanding something?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   Thanks again @pitrou, just a reminder  this PR was updated. Main change related to DropNullTable
   
   > ... is to get TableBatchReader to get a vector of record batches spanning the table, then call drop_null on each record batch and then Table::FromRecordBatches to get the resulting table.
   
   https://github.com/apache/arrow/pull/10802#discussion_r687883363


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);

Review comment:
       I think instead of explicitly building indices of valid bits, it'd be more compact and performant to reuse GetTakeIndices. You can zero-copy construct a BooleanArray which is true where `column` is valid and false where `column` is null, then pass it to GetTakeIndices to acquire `indices`




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa edited a comment on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   I updated this PR addressing feedback comments. 
   
   Main change, the following test cases were added.
   - 0-lengthed inputs (to test the early termination code paths)
   - non-zero but all null values on inputs (to test the early termination code paths)
   - Larger test case with more random values (say 1000 or so) --> Because bitmap ops work on 8 elements together. So, we need to test beyond 8 elements in an array.
   - Test with non-zero offsets in the inputs (for chunkedarrays and tables)
   
   cc @nirandaperera, @westonpace 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -591,6 +591,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def drop_null(data, memory_pool=None):

Review comment:
       I'll remove this as the proper definition is auto-generated from the C++ metadata.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,167 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::BooleanArray>> GetDropNullFilter(const Array& values,
+                                                               MemoryPool* memory_pool) {
+  auto bitmap_buffer = values.null_bitmap();
+  std::shared_ptr<arrow::BooleanArray> out_array = std::make_shared<BooleanArray>(
+      values.length(), bitmap_buffer, nullptr, 0, values.offset());
+  return out_array;
+}
+
+Result<std::shared_ptr<Array>> CreateEmptyArray(std::shared_ptr<DataType> type,
+                                                MemoryPool* memory_pool) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  return builder->Finish();
+}
+
+Result<std::shared_ptr<ChunkedArray>> CreateEmptyChunkedArray(
+    std::shared_ptr<DataType> type, MemoryPool* memory_pool) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_ASSIGN_OR_RAISE(new_chunks[0], CreateEmptyArray(type, memory_pool));
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<Datum> DropNullArray(const std::shared_ptr<Array>& values, ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->null_count() == values->length()) {
+    return CreateEmptyArray(values->type(), ctx->memory_pool());
+  }
+  if (values->type()->id() == Type::type::NA) {
+    return std::make_shared<NullArray>(0);
+  }
+  ARROW_ASSIGN_OR_RAISE(auto drop_null_filter,
+                        GetDropNullFilter(*values, ctx->memory_pool()));
+
+  auto options = FilterOptions::Defaults();
+  return CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)},
+                      &options, ctx);
+}
+
+Result<Datum> DropNullChunkedArray(const std::shared_ptr<ChunkedArray>& values,
+                                   ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->null_count() == values->length()) {
+    return CreateEmptyChunkedArray(values->type(), ctx->memory_pool());
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values->chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk.length() > 0) {
+      new_chunks.push_back(new_chunk.make_array());
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<Datum> DropNullRecordBatch(const std::shared_ptr<RecordBatch>& batch,
+                                  ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch->columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return batch;
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch->num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch->num_rows(), true);
+  for (const auto& column : batch->columns()) {
+    if (column->type()->id() == Type::type::NA) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch->num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch->num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch->num_rows()) {

Review comment:
       This condition is never satisfied, will fix.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,548 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+  CheckDropNull(boolean(), "[]", "[]");
+  CheckDropNull(boolean(), "[null, null]", "[]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  std::shared_ptr<Int32Array> Offsets(int32_t length, int32_t slice_count) {
+    return checked_pointer_cast<Int32Array>(rng_.Offsets(slice_count, 0, length));
+  }
+
+  ArrayVector Slices(const std::shared_ptr<Array>& array,
+                     const std::shared_ptr<Int32Array>& offsets) {
+    ArrayVector slices(offsets->length() - 1);
+    for (int64_t i = 0; i != static_cast<int64_t>(slices.size()); ++i) {
+      slices[i] =
+          array->Slice(offsets->Value(i), offsets->Value(i + 1) - offsets->Value(i));
+    }
+    return slices;
+  }
+
+  random::SeedType seed_ = 0xdeadbeef;
+  random::RandomArrayGenerator rng_;
+};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryArrowTypes);
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullString) {
+  this->AssertDropNull(R"(["a", "b", "c"])", R"(["a", "b", "c"])");
+  this->AssertDropNull(R"([null, "b", "c"])", "[\"b\", \"c\"]");
+  this->AssertDropNull(R"(["a", "b", null])", R"(["a", "b"])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(), R"(["a", "b", "c"])");
+}
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullDictionary) {
+  auto dict = R"(["a", "b", "c", "d", "e"])";
+  this->AssertDropNullDictionary(dict, "[3, 4, 2]", "[3, 4, 2]");
+  this->AssertDropNullDictionary(dict, "[null, 4, 2]", "[4, 2]");
+}
+
+class TestDropNullKernelFSB : public TestDropNullKernelTyped<FixedSizeBinaryType> {
+ public:
+  std::shared_ptr<DataType> value_type() { return fixed_size_binary(3); }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+};
+
+TEST_F(TestDropNullKernelFSB, DropNullFixedSizeBinary) {
+  this->AssertDropNull(R"(["aaa", "bbb", "ccc"])", R"(["aaa", "bbb", "ccc"])");
+  this->AssertDropNull(R"([null, "bbb", "ccc"])", "[\"bbb\", \"ccc\"]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(),
+                                                R"(["aaa", "bbb", "ccc"])");
+}
+
+class TestDropNullKernelWithList : public TestDropNullKernelTyped<ListType> {};
+
+TEST_F(TestDropNullKernelWithList, DropNullListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(list(int32()), list_json, "[[], [1,2], [3]]");
+  this->TestNoValidityBitmapButUnknownNullCount(list(int32()), "[[], [1,2], [3]]");
+}
+
+TEST_F(TestDropNullKernelWithList, DropNullListListInt32) {
+  std::string list_json = R"([
+    [],
+    [[1], [2, null, 2], []],
+    null,
+    [[3, null], null]
+  ])";
+  auto type = list(list(int32()));
+  CheckDropNull(type, list_json, R"([
+    [],
+    [[1], [2, null, 2], []],
+    [[3, null], null]
+  ])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(type,
+                                                "[[[1], [2, null, 2], []], [[3, null]]]");
+}
+
+class TestDropNullKernelWithLargeList : public TestDropNullKernelTyped<LargeListType> {};
+
+TEST_F(TestDropNullKernelWithLargeList, DropNullLargeListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(large_list(int32()), list_json, "[[], [1,2],  [3]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithFixedSizeList
+    : public TestDropNullKernelTyped<FixedSizeListType> {};
+
+TEST_F(TestDropNullKernelWithFixedSizeList, DropNullFixedSizeListInt32) {
+  std::string list_json = "[null, [1, null, 3], [4, 5, 6], [7, 8, null]]";
+  CheckDropNull(fixed_size_list(int32(), 3), list_json,
+                "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithMap : public TestDropNullKernelTyped<MapType> {};
+
+TEST_F(TestDropNullKernelWithMap, DropNullMapStringToInt32) {
+  std::string map_json = R"([
+    [["joe", 0], ["mark", null]],
+    null,
+    [["cap", 8]],
+    []
+  ])";
+  std::string expected_json = R"([
+    [["joe", 0], ["mark", null]],
+    [["cap", 8]],
+    []
+  ])";
+  CheckDropNull(map(utf8(), int32()), map_json, expected_json);
+}
+
+class TestDropNullKernelWithStruct : public TestDropNullKernelTyped<StructType> {};
+
+TEST_F(TestDropNullKernelWithStruct, DropNullStruct) {
+  auto struct_type = struct_({field("a", int32()), field("b", utf8())});
+  auto struct_json = R"([
+    null,
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  auto expected_struct_json = R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  CheckDropNull(struct_type, struct_json, expected_struct_json);
+  this->TestNoValidityBitmapButUnknownNullCount(struct_type, expected_struct_json);
+}
+
+class TestDropNullKernelWithUnion : public TestDropNullKernelTyped<UnionType> {};
+
+TEST_F(TestDropNullKernelWithUnion, DropNullUnion) {
+  auto union_type = dense_union({field("a", int32()), field("b", utf8())}, {2, 5});
+  auto union_json = R"([
+      [2, null],
+      [2, 222],
+      [5, "hello"],
+      [5, "eh"],
+      [2, null],
+      [2, 111],
+      [5, null]
+    ])";
+  CheckDropNull(union_type, union_json, union_json);
+}
+
+class TestDropNullKernelWithRecordBatch : public TestDropNullKernelTyped<RecordBatch> {
+ public:
+  void AssertDropNull(const std::shared_ptr<Schema>& schm, const std::string& batch_json,
+                      const std::string& expected_batch) {
+    std::shared_ptr<RecordBatch> actual;
+
+    ASSERT_OK(this->DoDropNull(schm, batch_json, &actual));
+    ValidateOutput(actual);
+    ASSERT_BATCHES_EQUAL(*RecordBatchFromJSON(schm, expected_batch), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<Schema>& schm, const std::string& batch_json,
+                    std::shared_ptr<RecordBatch>* out) {
+    auto batch = RecordBatchFromJSON(schm, batch_json);
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(batch));
+    *out = out_datum.record_batch();
+    return Status::OK();
+  }
+};
+
+TEST_F(TestDropNullKernelWithRecordBatch, DropNullRecordBatch) {
+  std::vector<std::shared_ptr<Field>> fields = {field("a", int32()), field("b", utf8())};
+  auto schm = schema(fields);
+
+  auto batch_json = R"([
+    {"a": null, "b": "yo"},
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  this->AssertDropNull(schm, batch_json, R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])");
+
+  batch_json = R"([
+    {"a": null, "b": "yo"},
+    {"a": 1, "b": null},
+    {"a": null, "b": "hello"},
+    {"a": 4, "b": null}
+  ])";
+  this->AssertDropNull(schm, batch_json, R"([])");
+  this->AssertDropNull(schm, R"([])", R"([])");
+}
+
+class TestDropNullKernelWithChunkedArray : public TestDropNullKernelTyped<ChunkedArray> {
+ public:
+  TestDropNullKernelWithChunkedArray()
+      : sizes_({0, 1, 2, 4, 16, 31, 1234}),
+        null_probabilities_({0.0, 0.1, 0.5, 0.9, 1.0}) {}
+
+  void AssertDropNull(const std::shared_ptr<DataType>& type,
+                      const std::vector<std::string>& values,
+                      const std::vector<std::string>& expected) {
+    std::shared_ptr<ChunkedArray> actual;
+    ASSERT_OK(this->DoDropNull(type, values, &actual));
+    ValidateOutput(actual);
+
+    AssertChunkedEqual(*ChunkedArrayFromJSON(type, expected), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<DataType>& type,
+                    const std::vector<std::string>& values,
+                    std::shared_ptr<ChunkedArray>* out) {
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(ChunkedArrayFromJSON(type, values)));
+    *out = out_datum.chunked_array();
+    return Status::OK();
+  }
+
+  template <typename ArrayFactory>
+  void CheckDropNullWithSlices(ArrayFactory&& factory) {
+    for (auto size : this->sizes_) {
+      for (auto null_probability : this->null_probabilities_) {
+        std::shared_ptr<Array> concatenated_array;
+        std::shared_ptr<ChunkedArray> chunked_array;
+        factory(size, null_probability, &chunked_array, &concatenated_array);
+
+        ASSERT_OK_AND_ASSIGN(auto out_datum, DropNull(chunked_array));
+        auto actual_chunked_array = out_datum.chunked_array();
+        ASSERT_OK_AND_ASSIGN(auto actual, Concatenate(actual_chunked_array->chunks()));
+
+        ASSERT_OK_AND_ASSIGN(out_datum, DropNull(*concatenated_array));
+        auto expected = out_datum.make_array();
+
+        AssertArraysEqual(*expected, *actual);
+      }
+    }
+  }
+
+  std::vector<int32_t> sizes_;
+  std::vector<double> null_probabilities_;
+};
+
+TEST_F(TestDropNullKernelWithChunkedArray, DropNullChunkedArray) {
+  this->AssertDropNull(int8(), {"[]"}, {"[]"});
+  this->AssertDropNull(int8(), {"[null]", "[8, null]"}, {"[8]"});
+
+  this->AssertDropNull(int8(), {"[null]", "[null, null]"}, {"[]"});
+  this->AssertDropNull(int8(), {"[7]", "[8, 9]"}, {"[7]", "[8, 9]"});
+  this->AssertDropNull(int8(), {"[]", "[]"}, {"[]", "[]"});
+}
+
+TEST_F(TestDropNullKernelWithChunkedArray, DropNullChunkedArrayWithSlices) {
+  // With Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<ChunkedArray>* out_chunked_array,
+                                       std::shared_ptr<Array>* out_concatenated_array) {
+    auto array = std::make_shared<NullArray>(size);
+    auto offsets = this->Offsets(size, 3);
+    auto slices = this->Slices(array, offsets);
+    *out_chunked_array = std::make_shared<ChunkedArray>(std::move(slices));
+
+    ASSERT_OK_AND_ASSIGN(*out_concatenated_array,
+                         Concatenate((*out_chunked_array)->chunks()));
+  });
+  // Without Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<ChunkedArray>* out_chunked_array,
+                                       std::shared_ptr<Array>* out_concatenated_array) {
+    auto array = this->rng_.ArrayOf(int16(), size, null_probability);
+    auto offsets = this->Offsets(size, 3);
+    auto slices = this->Slices(array, offsets);
+    *out_chunked_array = std::make_shared<ChunkedArray>(std::move(slices));
+
+    ASSERT_OK_AND_ASSIGN(*out_concatenated_array,
+                         Concatenate((*out_chunked_array)->chunks()));
+  });
+}
+
+class TestDropNullKernelWithTable : public TestDropNullKernelTyped<Table> {
+ public:
+  TestDropNullKernelWithTable()
+      : sizes_({0, 1, 2, 4, 16, 31, 1234}),
+        null_probabilities_({0.0, 0.1, 0.5, 0.9, 1.0}) {}
+
+  void AssertDropNull(const std::shared_ptr<Schema>& schm,
+                      const std::vector<std::string>& table_json,
+                      const std::vector<std::string>& expected_table) {
+    std::shared_ptr<Table> actual;
+    ASSERT_OK(this->DoDropNull(schm, table_json, &actual));
+    ValidateOutput(actual);
+    ASSERT_TABLES_EQUAL(*TableFromJSON(schm, expected_table), *actual);
+  }
+
+  Status DoDropNull(const std::shared_ptr<Schema>& schm,
+                    const std::vector<std::string>& values, std::shared_ptr<Table>* out) {
+    ARROW_ASSIGN_OR_RAISE(Datum out_datum, DropNull(TableFromJSON(schm, values)));
+    *out = out_datum.table();
+    return Status::OK();
+  }
+
+  template <typename ArrayFactory>
+  void CheckDropNullWithSlices(ArrayFactory&& factory) {
+    for (auto size : this->sizes_) {
+      for (auto null_probability : this->null_probabilities_) {
+        std::shared_ptr<Table> table_w_slices;
+        std::shared_ptr<Table> table_wo_slices;
+
+        factory(size, null_probability, &table_w_slices, &table_wo_slices);
+
+        ASSERT_OK_AND_ASSIGN(auto out_datum, DropNull(table_w_slices));
+        auto actual = out_datum.table();
+
+        ASSERT_OK_AND_ASSIGN(out_datum, DropNull(table_wo_slices));
+        auto expected = out_datum.table();
+        if (actual->num_rows() > 0) {
+          ASSERT_TRUE(actual->num_rows() == expected->num_rows());
+          for (int index = 0; index < actual->num_columns(); index++) {
+            ASSERT_OK_AND_ASSIGN(auto actual_col,
+                                 Concatenate(actual->column(index)->chunks()));
+            ASSERT_OK_AND_ASSIGN(auto expected_col,
+                                 Concatenate(expected->column(index)->chunks()));
+            AssertArraysEqual(*actual_col, *expected_col);
+          }
+        }
+      }
+    }
+  }
+
+  std::vector<int32_t> sizes_;
+  std::vector<double> null_probabilities_;
+};
+
+TEST_F(TestDropNullKernelWithTable, DropNullTable) {
+  std::vector<std::shared_ptr<Field>> fields = {field("a", int32()), field("b", utf8())};
+  auto schm = schema(fields);
+
+  {
+    std::vector<std::string> table_json = {R"([
+      {"a": null, "b": "yo"},
+      {"a": 1, "b": ""}
+    ])",
+                                           R"([
+      {"a": 2, "b": "hello"},
+      {"a": 4, "b": "eh"}
+    ])"};
+    std::vector<std::string> expected_table_json = {R"([
+      {"a": 1, "b": ""}
+    ])",
+                                                    R"([
+      {"a": 2, "b": "hello"},
+      {"a": 4, "b": "eh"}
+    ])"};
+    this->AssertDropNull(schm, table_json, expected_table_json);
+  }
+  {
+    std::vector<std::string> table_json = {R"([
+      {"a": null, "b": "yo"},
+      {"a": 1, "b": null}
+    ])",
+                                           R"([
+      {"a": 2, "b": null},
+      {"a": null, "b": "eh"}
+    ])"};
+    std::shared_ptr<Table> actual;
+    ASSERT_OK(this->DoDropNull(schm, table_json, &actual));
+    ASSERT_EQ(actual->num_rows(), 0);
+  }
+}
+
+TEST_F(TestDropNullKernelWithTable, DropNullTableWithWithSlices) {
+  // With Null Arrays
+  this->CheckDropNullWithSlices([this](int32_t size, double null_probability,
+                                       std::shared_ptr<Table>* out_table_w_slices,
+                                       std::shared_ptr<Table>* out_table_wo_slices) {
+    std::vector<std::shared_ptr<Field>> fields = {field("a", int32()),
+                                                  field("b", utf8())};
+    auto schm = schema(fields);
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(fields.size());
+    auto col_a = std::make_shared<NullArray>(size);
+    auto col_b = std::make_shared<NullArray>(size);

Review comment:
       These are the wrong column types. I will fix and add a validation.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] commented on pull request #10802: ARROW-1568: [ComputeKernels] Add Drop Null for Arrays Kernel

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #10802:
URL: https://github.com/apache/arrow/pull/10802#issuecomment-886805671


   https://issues.apache.org/jira/browse/ARROW-1568


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,211 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::BooleanArray>> GetDropNullFilter(const Array& values,
+                                                               MemoryPool* memory_pool) {
+  auto bitmap_buffer = values.null_bitmap();
+  std::shared_ptr<arrow::BooleanArray> out_array = std::make_shared<BooleanArray>(
+      values.length(), bitmap_buffer, nullptr, 0, values.offset());
+  return out_array;
+}
+
+Result<std::shared_ptr<Array>> CreateEmptyArray(std::shared_ptr<DataType> type,
+                                                MemoryPool* memory_pool) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  return builder->Finish();
+}
+
+Result<std::shared_ptr<ChunkedArray>> CreateEmptyChunkedArray(
+    std::shared_ptr<DataType> type, MemoryPool* memory_pool) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_ASSIGN_OR_RAISE(new_chunks[0], CreateEmptyArray(type, memory_pool));
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->null_count() == values->length()) {
+    return CreateEmptyArray(values->type(), ctx->memory_pool());
+  }
+  if (values->type()->id() == Type::type::NA) {
+    return std::make_shared<NullArray>(0);
+  }
+  ARROW_ASSIGN_OR_RAISE(auto drop_null_filter,
+                        GetDropNullFilter(*values, ctx->memory_pool()));
+
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    return CreateEmptyChunkedArray(values.type(), ctx->memory_pool());
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->id() == Type::type::NA) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());

Review comment:
       No need to change this, but note we have `ArrayVector` and other similar aliases (such as `BufferVector` etc.) which may ease typing.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] westonpace commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+  for (const auto& column : batch.columns()) {
+    if (column->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  int64_t null_count = 0;
+  for (const auto& col : table.columns()) {
+    for (const auto& column_chunk : col->chunks()) {
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (const auto& col : table.columns()) {
+    if (col->type()->Equals(arrow::null())) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), false);
+      break;
+    }
+    std::vector<::arrow::internal::Bitmap> bitmaps;
+    std::transform(col->chunks().begin(), col->chunks().end(),
+                   std::back_inserter(bitmaps), [](const std::shared_ptr<Array>& array) {
+                     return ::arrow::internal::Bitmap(array->null_bitmap_data(),
+                                                      array->offset(), array->length());
+                   });
+    int64_t global_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), global_offset);
+      }
+      global_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(table.num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(table), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.table();
+}
+
+const FunctionDoc drop_null_doc(
+    "Drop nulls from the input",
+    ("The output is populated with values from the input (Array, ChunkedArray, "
+     "RecordBatch, or Table) without the null values."
+     "Note that for the RecordBatch/Table cases, `drop_null` drops the full row if there "
+     "is any null"
+     "null."),

Review comment:
       ```suggestion
        "is any null."),
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }
+      bitmap_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }

Review comment:
       Nice! Thanks Niranda




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,182 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr,
+                                              values.null_count(), values.offset());

Review comment:
       ```suggestion
                                                 0, values.offset());
   ```
   Once the validity buffer is promoted to a data buffer, we should consider it as a non-null array.   `values.null_count() != 0`, so, we can simply pass 0 here. 

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,

Review comment:
       well, it's not actually about the offset. What would be the output of the above example? An empty table?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1718,5 +1718,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryTypes);
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullString) {
+  this->AssertDropNull(R"(["a", "b", "c"])", R"(["a", "b", "c"])");
+  this->AssertDropNull(R"([null, "b", "c"])", "[\"b\", \"c\"]");
+  this->AssertDropNull(R"(["a", "b", null])", R"(["a", "b"])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(), R"(["a", "b", "c"])");
+}
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullDictionary) {
+  auto dict = R"(["a", "b", "c", "d", "e"])";
+  this->AssertDropNullDictionary(dict, "[3, 4, 2]", "[3, 4, 2]");
+  this->AssertDropNullDictionary(dict, "[null, 4, 2]", "[4, 2]");
+}
+
+class TestDropNullKernelFSB : public TestDropNullKernelTyped<FixedSizeBinaryType> {
+ public:
+  std::shared_ptr<DataType> value_type() { return fixed_size_binary(3); }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+};
+
+TEST_F(TestDropNullKernelFSB, DropNullFixedSizeBinary) {
+  this->AssertDropNull(R"(["aaa", "bbb", "ccc"])", R"(["aaa", "bbb", "ccc"])");
+  this->AssertDropNull(R"([null, "bbb", "ccc"])", "[\"bbb\", \"ccc\"]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(),
+                                                R"(["aaa", "bbb", "ccc"])");
+}
+
+class TestDropNullKernelWithList : public TestDropNullKernelTyped<ListType> {};
+
+TEST_F(TestDropNullKernelWithList, DropNullListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(list(int32()), list_json, "[[], [1,2], [3]]");
+  this->TestNoValidityBitmapButUnknownNullCount(list(int32()), "[[], [1,2], [3]]");
+}
+
+TEST_F(TestDropNullKernelWithList, DropNullListListInt32) {
+  std::string list_json = R"([
+    [],
+    [[1], [2, null, 2], []],
+    null,
+    [[3, null], null]
+  ])";
+  auto type = list(list(int32()));
+  CheckDropNull(type, list_json, R"([
+    [],
+    [[1], [2, null, 2], []],
+    [[3, null], null]
+  ])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(type,
+                                                "[[[1], [2, null, 2], []], [[3, null]]]");
+}
+
+class TestDropNullKernelWithLargeList : public TestDropNullKernelTyped<LargeListType> {};
+
+TEST_F(TestDropNullKernelWithLargeList, DropNullLargeListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(large_list(int32()), list_json, "[[], [1,2],  [3]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithFixedSizeList
+    : public TestDropNullKernelTyped<FixedSizeListType> {};
+
+TEST_F(TestDropNullKernelWithFixedSizeList, DropNullFixedSizeListInt32) {
+  std::string list_json = "[null, [1, null, 3], [4, 5, 6], [7, 8, null]]";
+  CheckDropNull(fixed_size_list(int32(), 3), list_json,
+                "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithMap : public TestDropNullKernelTyped<MapType> {};
+
+TEST_F(TestDropNullKernelWithMap, DropNullMapStringToInt32) {
+  std::string map_json = R"([
+    [["joe", 0], ["mark", null]],
+    null,
+    [["cap", 8]],
+    []
+  ])";
+  std::string expected_json = R"([
+    [["joe", 0], ["mark", null]],
+    [["cap", 8]],
+    []
+  ])";
+  CheckDropNull(map(utf8(), int32()), map_json, expected_json);
+}
+
+class TestDropNullKernelWithStruct : public TestDropNullKernelTyped<StructType> {};
+
+TEST_F(TestDropNullKernelWithStruct, DropNullStruct) {
+  auto struct_type = struct_({field("a", int32()), field("b", utf8())});
+  auto struct_json = R"([
+    null,
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  auto expected_struct_json = R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  CheckDropNull(struct_type, struct_json, expected_struct_json);
+  this->TestNoValidityBitmapButUnknownNullCount(struct_type, expected_struct_json);
+}
+
+class TestDropNullKernelWithUnion : public TestDropNullKernelTyped<UnionType> {};
+
+TEST_F(TestDropNullKernelWithUnion, DropNullUnion) {
+  auto union_type = dense_union({field("a", int32()), field("b", utf8())}, {2, 5});
+  auto union_json = R"([
+      [2, null],
+      [2, 222],
+      [5, "hello"],
+      [5, "eh"],
+      [2, null],
+      [2, 111],
+      [5, null]
+    ])";
+  CheckDropNull(union_type, union_json, union_json);
+}
+

Review comment:
       AFAIK, we have utils build for Scalar kernels. But not many are available for vector kernels (and one main reason is, in vector kernels there are a lot of moving pieces). 
   
   I'd like to suggest that we add a test case with random data (say 1000 rows), so that we know we go beyond the byte boundaries (for bitmaps), etc. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   This PR is ready to review :) 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_RETURN_NOT_OK(CreateEmptyArray(type, memory_pool, &new_chunks[0]));
+  *output_array = std::make_shared<ChunkedArray>(std::move(new_chunks));
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  if (values.null_count() == values.length()) {
+    std::shared_ptr<ChunkedArray> empty_array;
+    RETURN_NOT_OK(
+        CreateEmptyChunkedArray(values.type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values.chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk->length() > 0) {
+      new_chunks.push_back(new_chunk);
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch.columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());

Review comment:
       Thanks I didn't realize that this was not changed. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] westonpace commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -1988,7 +1989,7 @@ class FilterMetaFunction : public MetaFunction {
 
 Result<std::shared_ptr<Array>> TakeAA(const Array& values, const Array& indices,
                                       const TakeOptions& options, ExecContext* ctx) {
-  ARROW_ASSIGN_OR_RAISE(Datum result,
+    ARROW_ASSIGN_OR_RAISE(Datum result,

Review comment:
       What's going on here?  I've found it easiest to configure the IDE to run format-on-save.

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,51 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_dropnull(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.dropnull()
+    result.validate()
+    indices = [i for i in range(len(arr)) if arr[i].is_valid]
+    expected = arr.take(pa.array(indices))
+    assert result.equals(expected)
+
+
+def test_dropnull_chunked_array():
+    arr = pa.chunked_array([["a", None], ["c", "d", None]])
+    expected_drop = pa.chunked_array([["a"], ["c", "d"]])
+    result = arr.dropnull()
+    assert result.equals(expected_drop)
+
+
+def test_dropnull_record_batch():
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None])], names=["a'"])

Review comment:
       Can you make some tests for record batches with multiple columns? (I think this will fail if they have different numbers of null elements).

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t length_count = 0;
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  for (int i = 0; i < batch.num_columns(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(auto indices,
+                          GetNotNullIndices(batch.column(i), ctx->memory_pool()));
+    ARROW_ASSIGN_OR_RAISE(Datum out, Take(batch.column(i)->data(), Datum(indices),
+                                          TakeOptions::NoBoundsCheck(), ctx));
+    columns[i] = out.make_array();
+    length_count += columns[i]->length();
+  }
+  return RecordBatch::Make(batch.schema(), length_count, std::move(columns));
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(relative_index + i);
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  std::shared_ptr<arrow::Array> indices;

Review comment:
       Can you add a check here.  If the length of `notnull_indices` is equal to the length of the table then you can just return the table an save some time.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1718,5 +1718,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryTypes);
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullString) {
+  this->AssertDropNull(R"(["a", "b", "c"])", R"(["a", "b", "c"])");
+  this->AssertDropNull(R"([null, "b", "c"])", "[\"b\", \"c\"]");
+  this->AssertDropNull(R"(["a", "b", null])", R"(["a", "b"])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(), R"(["a", "b", "c"])");
+}
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullDictionary) {
+  auto dict = R"(["a", "b", "c", "d", "e"])";
+  this->AssertDropNullDictionary(dict, "[3, 4, 2]", "[3, 4, 2]");
+  this->AssertDropNullDictionary(dict, "[null, 4, 2]", "[4, 2]");
+}
+
+class TestDropNullKernelFSB : public TestDropNullKernelTyped<FixedSizeBinaryType> {
+ public:
+  std::shared_ptr<DataType> value_type() { return fixed_size_binary(3); }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+};
+
+TEST_F(TestDropNullKernelFSB, DropNullFixedSizeBinary) {
+  this->AssertDropNull(R"(["aaa", "bbb", "ccc"])", R"(["aaa", "bbb", "ccc"])");
+  this->AssertDropNull(R"([null, "bbb", "ccc"])", "[\"bbb\", \"ccc\"]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(),
+                                                R"(["aaa", "bbb", "ccc"])");
+}
+
+class TestDropNullKernelWithList : public TestDropNullKernelTyped<ListType> {};
+
+TEST_F(TestDropNullKernelWithList, DropNullListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(list(int32()), list_json, "[[], [1,2], [3]]");
+  this->TestNoValidityBitmapButUnknownNullCount(list(int32()), "[[], [1,2], [3]]");
+}
+
+TEST_F(TestDropNullKernelWithList, DropNullListListInt32) {
+  std::string list_json = R"([
+    [],
+    [[1], [2, null, 2], []],
+    null,
+    [[3, null], null]
+  ])";
+  auto type = list(list(int32()));
+  CheckDropNull(type, list_json, R"([
+    [],
+    [[1], [2, null, 2], []],
+    [[3, null], null]
+  ])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(type,
+                                                "[[[1], [2, null, 2], []], [[3, null]]]");
+}
+
+class TestDropNullKernelWithLargeList : public TestDropNullKernelTyped<LargeListType> {};
+
+TEST_F(TestDropNullKernelWithLargeList, DropNullLargeListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(large_list(int32()), list_json, "[[], [1,2],  [3]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithFixedSizeList
+    : public TestDropNullKernelTyped<FixedSizeListType> {};
+
+TEST_F(TestDropNullKernelWithFixedSizeList, DropNullFixedSizeListInt32) {
+  std::string list_json = "[null, [1, null, 3], [4, 5, 6], [7, 8, null]]";
+  CheckDropNull(fixed_size_list(int32(), 3), list_json,
+                "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithMap : public TestDropNullKernelTyped<MapType> {};
+
+TEST_F(TestDropNullKernelWithMap, DropNullMapStringToInt32) {
+  std::string map_json = R"([
+    [["joe", 0], ["mark", null]],
+    null,
+    [["cap", 8]],
+    []
+  ])";
+  std::string expected_json = R"([
+    [["joe", 0], ["mark", null]],
+    [["cap", 8]],
+    []
+  ])";
+  CheckDropNull(map(utf8(), int32()), map_json, expected_json);
+}
+
+class TestDropNullKernelWithStruct : public TestDropNullKernelTyped<StructType> {};
+
+TEST_F(TestDropNullKernelWithStruct, DropNullStruct) {
+  auto struct_type = struct_({field("a", int32()), field("b", utf8())});
+  auto struct_json = R"([
+    null,
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  auto expected_struct_json = R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  CheckDropNull(struct_type, struct_json, expected_struct_json);
+  this->TestNoValidityBitmapButUnknownNullCount(struct_type, expected_struct_json);
+}
+
+class TestDropNullKernelWithUnion : public TestDropNullKernelTyped<UnionType> {};
+
+TEST_F(TestDropNullKernelWithUnion, DropNullUnion) {
+  auto union_type = dense_union({field("a", int32()), field("b", utf8())}, {2, 5});
+  auto union_json = R"([
+      [2, null],
+      [2, 222],
+      [5, "hello"],
+      [5, "eh"],
+      [2, null],
+      [2, 111],
+      [5, null]
+    ])";
+  CheckDropNull(union_type, union_json, union_json);
+}
+

Review comment:
       I don't know what the rules are for what parts get tested in C++ and what parts get tested in JSON.  @nirandaperera / @bkietz maybe you can help me out here.  However, there are no C++ tests for the record batch, chunked array, or table versionsof drop null.  Though maybe this is ok.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t length_count = 0;
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  for (int i = 0; i < batch.num_columns(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(auto indices,
+                          GetNotNullIndices(batch.column(i), ctx->memory_pool()));
+    ARROW_ASSIGN_OR_RAISE(Datum out, Take(batch.column(i)->data(), Datum(indices),
+                                          TakeOptions::NoBoundsCheck(), ctx));
+    columns[i] = out.make_array();
+    length_count += columns[i]->length();
+  }
+  return RecordBatch::Make(batch.schema(), length_count, std::move(columns));
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  std::vector<ArrayVector> inputs(num_columns);
+
+  // Fetch table columns
+  for (int i = 0; i < num_columns; ++i) {
+    inputs[i] = table.column(i)->chunks();
+  }
+  std::set<int32_t> notnull_indices;
+  // Rechunk inputs to allow consistent iteration over their respective chunks
+  inputs = arrow::internal::RechunkArraysConsistently(inputs);
+
+  const int64_t num_chunks = static_cast<int64_t>(inputs.back().size());
+  for (int col = 0; col < num_columns; ++col) {
+    int64_t relative_index = 0;
+    for (int64_t chunk_index = 0; chunk_index < num_chunks; ++chunk_index) {
+      const auto& column_chunk = inputs[col][chunk_index];
+      for (int64_t i = 0; i < column_chunk->length(); ++i) {
+        if (!column_chunk->IsValid(i)) {
+          notnull_indices.insert(relative_index + i);
+        }
+      }
+      relative_index += column_chunk->length();
+    }
+  }
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(ctx->memory_pool());
+  builder.Reserve(static_cast<int64_t>(table.num_rows() - notnull_indices.size()));
+  for (int64_t row_index = 0; row_index < table.num_rows(); ++row_index) {
+    if (notnull_indices.find(row_index) == notnull_indices.end()) {
+      builder.UnsafeAppend(static_cast<int32_t>(row_index));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return TakeTA(table, *indices, TakeOptions::Defaults(), ctx);
+}
+
+const FunctionDoc dropnull_doc(
+    "DropNull kernel",
+    ("The output is populated with values from the input without the null values"),
+    {"input"});
+
+class DropNullMetaFunction : public MetaFunction {
+ public:
+  DropNullMetaFunction() : MetaFunction("dropnull", Arity::Unary(), &dropnull_doc) {}
+
+  Result<Datum> ExecuteImpl(const std::vector<Datum>& args,
+                            const FunctionOptions* options,
+                            ExecContext* ctx) const override {
+    switch (args[0].kind()) {
+      case Datum::ARRAY: {
+        const auto values = args[0].make_array();
+        ARROW_ASSIGN_OR_RAISE(auto indices,

Review comment:
       If `values.null_count` is known (and 0) can we skip some work here?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(column->length() - column->null_count());
+
+  std::vector<int32_t> values;
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.UnsafeAppend(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  builder.Reserve(chunks->length() - chunks->null_count());
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);
+    for (int64_t col_index = 0; col_index < column_chunk->length(); col_index++) {
+      if (column_chunk->IsValid(col_index)) {
+        builder.UnsafeAppend(static_cast<int32_t>(relative_index + col_index));
+      }
+    }
+    relative_index += column_chunk->length();
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t length_count = 0;
+  std::vector<std::shared_ptr<Array>> columns(batch.num_columns());
+  for (int i = 0; i < batch.num_columns(); ++i) {
+    ARROW_ASSIGN_OR_RAISE(auto indices,
+                          GetNotNullIndices(batch.column(i), ctx->memory_pool()));
+    ARROW_ASSIGN_OR_RAISE(Datum out, Take(batch.column(i)->data(), Datum(indices),
+                                          TakeOptions::NoBoundsCheck(), ctx));
+    columns[i] = out.make_array();
+    length_count += columns[i]->length();
+  }
+  return RecordBatch::Make(batch.schema(), length_count, std::move(columns));
+}
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {

Review comment:
       If this check makes sense here then should we have the same check for the record batch and array versions?

##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1718,5 +1718,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);

Review comment:
       Since the implementation of DropNull relies on the Take kernel and there aren't different versions of it for each data type I don't know how much we need to test drop null on all the different types.
   
   @nirandaperera / @bkietz I'd be interested in a second opinion here as I haven't spent a lot of time in kernels code.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,144 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);

Review comment:
       If you had such a boolean array (and this may be future work) could it be more efficient to have a take that accepts a boolean array (of the same size as the input) and doesn't even construct the intermediate indices array?

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,51 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_dropnull(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.dropnull()
+    result.validate()
+    indices = [i for i in range(len(arr)) if arr[i].is_valid]
+    expected = arr.take(pa.array(indices))
+    assert result.equals(expected)
+
+
+def test_dropnull_chunked_array():
+    arr = pa.chunked_array([["a", None], ["c", "d", None]])
+    expected_drop = pa.chunked_array([["a"], ["c", "d"]])
+    result = arr.dropnull()
+    assert result.equals(expected_drop)
+
+
+def test_dropnull_record_batch():
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None])], names=["a'"])
+
+    result = batch.dropnull()
+    expected = pa.record_batch([pa.array(["a", "c", "d"])], names=["a'"])
+    assert result.equals(expected)
+
+
+def test_dropnull_table():
+    table = pa.table([pa.array(["a", None, "c", "d", None])], names=["a"])

Review comment:
       Can you make some tests for tables with multiple columns?




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def dropnull(data, *, memory_pool=None):

Review comment:
       I think the explicit function can thus be removed.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1718,5 +1718,240 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryTypes);
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullString) {
+  this->AssertDropNull(R"(["a", "b", "c"])", R"(["a", "b", "c"])");
+  this->AssertDropNull(R"([null, "b", "c"])", "[\"b\", \"c\"]");
+  this->AssertDropNull(R"(["a", "b", null])", R"(["a", "b"])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(), R"(["a", "b", "c"])");
+}
+
+TYPED_TEST(TestDropNullKernelWithString, DropNullDictionary) {
+  auto dict = R"(["a", "b", "c", "d", "e"])";
+  this->AssertDropNullDictionary(dict, "[3, 4, 2]", "[3, 4, 2]");
+  this->AssertDropNullDictionary(dict, "[null, 4, 2]", "[4, 2]");
+}
+
+class TestDropNullKernelFSB : public TestDropNullKernelTyped<FixedSizeBinaryType> {
+ public:
+  std::shared_ptr<DataType> value_type() { return fixed_size_binary(3); }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+};
+
+TEST_F(TestDropNullKernelFSB, DropNullFixedSizeBinary) {
+  this->AssertDropNull(R"(["aaa", "bbb", "ccc"])", R"(["aaa", "bbb", "ccc"])");
+  this->AssertDropNull(R"([null, "bbb", "ccc"])", "[\"bbb\", \"ccc\"]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(this->value_type(),
+                                                R"(["aaa", "bbb", "ccc"])");
+}
+
+class TestDropNullKernelWithList : public TestDropNullKernelTyped<ListType> {};
+
+TEST_F(TestDropNullKernelWithList, DropNullListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(list(int32()), list_json, "[[], [1,2], [3]]");
+  this->TestNoValidityBitmapButUnknownNullCount(list(int32()), "[[], [1,2], [3]]");
+}
+
+TEST_F(TestDropNullKernelWithList, DropNullListListInt32) {
+  std::string list_json = R"([
+    [],
+    [[1], [2, null, 2], []],
+    null,
+    [[3, null], null]
+  ])";
+  auto type = list(list(int32()));
+  CheckDropNull(type, list_json, R"([
+    [],
+    [[1], [2, null, 2], []],
+    [[3, null], null]
+  ])");
+
+  this->TestNoValidityBitmapButUnknownNullCount(type,
+                                                "[[[1], [2, null, 2], []], [[3, null]]]");
+}
+
+class TestDropNullKernelWithLargeList : public TestDropNullKernelTyped<LargeListType> {};
+
+TEST_F(TestDropNullKernelWithLargeList, DropNullLargeListInt32) {
+  std::string list_json = "[[], [1,2], null, [3]]";
+  CheckDropNull(large_list(int32()), list_json, "[[], [1,2],  [3]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithFixedSizeList
+    : public TestDropNullKernelTyped<FixedSizeListType> {};
+
+TEST_F(TestDropNullKernelWithFixedSizeList, DropNullFixedSizeListInt32) {
+  std::string list_json = "[null, [1, null, 3], [4, 5, 6], [7, 8, null]]";
+  CheckDropNull(fixed_size_list(int32(), 3), list_json,
+                "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+
+  this->TestNoValidityBitmapButUnknownNullCount(
+      fixed_size_list(int32(), 3), "[[1, null, 3], [4, 5, 6], [7, 8, null]]");
+}
+
+class TestDropNullKernelWithMap : public TestDropNullKernelTyped<MapType> {};
+
+TEST_F(TestDropNullKernelWithMap, DropNullMapStringToInt32) {
+  std::string map_json = R"([
+    [["joe", 0], ["mark", null]],
+    null,
+    [["cap", 8]],
+    []
+  ])";
+  std::string expected_json = R"([
+    [["joe", 0], ["mark", null]],
+    [["cap", 8]],
+    []
+  ])";
+  CheckDropNull(map(utf8(), int32()), map_json, expected_json);
+}
+
+class TestDropNullKernelWithStruct : public TestDropNullKernelTyped<StructType> {};
+
+TEST_F(TestDropNullKernelWithStruct, DropNullStruct) {
+  auto struct_type = struct_({field("a", int32()), field("b", utf8())});
+  auto struct_json = R"([
+    null,
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  auto expected_struct_json = R"([
+    {"a": 1, "b": ""},
+    {"a": 2, "b": "hello"},
+    {"a": 4, "b": "eh"}
+  ])";
+  CheckDropNull(struct_type, struct_json, expected_struct_json);
+  this->TestNoValidityBitmapButUnknownNullCount(struct_type, expected_struct_json);
+}
+
+class TestDropNullKernelWithUnion : public TestDropNullKernelTyped<UnionType> {};
+
+TEST_F(TestDropNullKernelWithUnion, DropNullUnion) {
+  auto union_type = dense_union({field("a", int32()), field("b", utf8())}, {2, 5});
+  auto union_json = R"([
+      [2, null],
+      [2, 222],
+      [5, "hello"],
+      [5, "eh"],
+      [2, null],
+      [2, 111],
+      [5, null]
+    ])";
+  CheckDropNull(union_type, union_json, union_json);
+}
+

Review comment:
       Test for record batch, chunked array and table are covered in the python test. IMO the c++ tests should be covered in another PR. 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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






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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   LGTM. +1 @aocsa Thank you for doing this! :-) 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] jorisvandenbossche commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/compute.py
##########
@@ -590,6 +590,38 @@ def take(data, indices, *, boundscheck=True, memory_pool=None):
     return call_function('take', [data, indices], options, memory_pool)
 
 
+def dropnull(data, *, memory_pool=None):

Review comment:
       I quickly tried it out, and also without this explicit function, the auto-generated one seems to work fine. 
   
   cc @pitrou do you know on what basis it's decided when to use the auto-generated one and when to add an explicit 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.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] westonpace commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}

Review comment:
       This seems like a very useful utility function.  I wonder if we want to put it somewhere more visible.  There could be a CreateEmptyRecordBatch and CreateEmptyTable too.  Can be done in a follow-up PR.

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }

Review comment:
       This can be done in a follow-up but I wonder if this copy could be avoided by passing offsets to BitmapAnd?

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,83 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_drop_null(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.drop_null()
+    result.validate()
+    indices = [i for i in range(len(arr)) if arr[i].is_valid]
+    expected = arr.take(pa.array(indices))
+    assert result.equals(expected)
+
+
+def test_drop_null_chunked_array():
+    arr = pa.chunked_array([["a", None], ["c", "d", None]])
+    expected_drop = pa.chunked_array([["a"], ["c", "d"]])
+    result = arr.drop_null()
+    assert result.equals(expected_drop)
+
+
+def test_drop_null_record_batch():
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None])], names=["a'"])
+    result = batch.drop_null()
+    expected = pa.record_batch([pa.array(["a", "c", "d"])], names=["a'"])
+    assert result.equals(expected)
+
+    batch = pa.record_batch(
+        [pa.array(["a", None, "c", "d", None]),
+         pa.array([None, None, "c", None, "e"])], names=["a'", "b'"])
+
+    result = batch.drop_null()
+    expected = pa.record_batch(
+        [pa.array(["c"]), pa.array(["c"])], names=["a'", "b'"])
+    print(result["a'"])
+    print(expected["a'"])

Review comment:
       Remove these prints

##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;

Review comment:
       Nit: Put this in the method or at the top of the namespace declaration.

##########
File path: python/pyarrow/tests/test_compute.py
##########
@@ -974,6 +974,83 @@ def test_take_null_type():
     assert len(table.take(indices).column(0)) == 4
 
 
+@pytest.mark.parametrize(('ty', 'values'), all_array_types)
+def test_drop_null(ty, values):
+    arr = pa.array(values, type=ty)
+    result = arr.drop_null()
+    result.validate()
+    indices = [i for i in range(len(arr)) if arr[i].is_valid]
+    expected = arr.take(pa.array(indices))
+    assert result.equals(expected)
+
+
+def test_drop_null_chunked_array():
+    arr = pa.chunked_array([["a", None], ["c", "d", None]])
+    expected_drop = pa.chunked_array([["a"], ["c", "d"]])

Review comment:
       ```suggestion
       arr = pa.chunked_array([["a", None], ["c", "d", None], [None], []])
       expected_drop = pa.chunked_array([["a"], ["c", "d"], [], []])
   ```




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   One last comment: can you document the function in `docs/source/cpp/compute.rst`?


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Status CreateEmptyChunkedArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                               std::shared_ptr<ChunkedArray>* output_array) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now

Review comment:
       Yes, it is allowed, but for consistency with Take:
   
   https://github.com/apache/arrow/blob/5c5a0d63a42dc8d5ecab5996574c466f2e9c2ed5/cpp/src/arrow/compute/kernels/vector_selection.cc#L2001
   
   And with  ChunkedArray::Make  
   
   https://github.com/apache/arrow/blob/5c5a0d63a42dc8d5ecab5996574c466f2e9c2ed5/cpp/src/arrow/chunked_array.cc#L72
   
   This is preferred to be consistent with the expected output results.
   




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}

Review comment:
       Nice!




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: python/pyarrow/table.pxi
##########
@@ -956,6 +963,13 @@ cdef class RecordBatch(_PandasConvertible):
         """
         return _pc().take(self, indices)
 
+    def drop_null(self):
+        """
+        Remove missing values from an RecordBatch.

Review comment:
       Ping.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] nirandaperera commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}

Review comment:
       Yep! We just had this discussion in Zulip. 
   https://ursalabs.zulipchat.com/#narrow/stream/271283-help.2Fc.2B.2B/topic/creating.20empty.20array.2F.20table.20using.20Slice
   
   https://issues.apache.org/jira/browse/ARROW-7051




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   Note the auto-generated Python  docstring isn't entirely accurate (see the `input` argument description):
   ```
   Signature: pc.drop_null(input, *, memory_pool=None)
   Docstring:
   Drop nulls from the input.
   
   The output is populated with values from the input (Array, ChunkedArray,
   RecordBatch, or Table) without the null values.
   For the RecordBatch and Table cases, `drop_null` drops the full row if
   there is any null.
   
   Parameters
   ----------
   input : Array-like or scalar-like
       Argument to compute function
   memory_pool : pyarrow.MemoryPool, optional
       If not passed, will allocate memory from the default memory pool.
   File:      ~/arrow/dev/python/pyarrow/compute.py
   Type:      function
   ```
   
   I'll open a separate JIRA.


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,219 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,

Review comment:
       previous to this I was using `Result<...>` for this helper function, Niranda suggest me this way instead. So I think this is just a preference. Personally I prefer the use of `Result<...>`




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] pitrou commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,167 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::BooleanArray>> GetDropNullFilter(const Array& values,
+                                                               MemoryPool* memory_pool) {
+  auto bitmap_buffer = values.null_bitmap();
+  std::shared_ptr<arrow::BooleanArray> out_array = std::make_shared<BooleanArray>(
+      values.length(), bitmap_buffer, nullptr, 0, values.offset());
+  return out_array;
+}
+
+Result<std::shared_ptr<Array>> CreateEmptyArray(std::shared_ptr<DataType> type,
+                                                MemoryPool* memory_pool) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  return builder->Finish();
+}
+
+Result<std::shared_ptr<ChunkedArray>> CreateEmptyChunkedArray(
+    std::shared_ptr<DataType> type, MemoryPool* memory_pool) {
+  std::vector<std::shared_ptr<Array>> new_chunks(1);  // Hard-coded 1 for now
+  ARROW_ASSIGN_OR_RAISE(new_chunks[0], CreateEmptyArray(type, memory_pool));
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<Datum> DropNullArray(const std::shared_ptr<Array>& values, ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->null_count() == values->length()) {
+    return CreateEmptyArray(values->type(), ctx->memory_pool());
+  }
+  if (values->type()->id() == Type::type::NA) {
+    return std::make_shared<NullArray>(0);
+  }
+  ARROW_ASSIGN_OR_RAISE(auto drop_null_filter,
+                        GetDropNullFilter(*values, ctx->memory_pool()));
+
+  auto options = FilterOptions::Defaults();
+  return CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)},
+                      &options, ctx);
+}
+
+Result<Datum> DropNullChunkedArray(const std::shared_ptr<ChunkedArray>& values,
+                                   ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->null_count() == values->length()) {
+    return CreateEmptyChunkedArray(values->type(), ctx->memory_pool());
+  }
+  std::vector<std::shared_ptr<Array>> new_chunks;
+  for (const auto& chunk : values->chunks()) {
+    ARROW_ASSIGN_OR_RAISE(auto new_chunk, DropNullArray(chunk, ctx));
+    if (new_chunk.length() > 0) {
+      new_chunks.push_back(new_chunk.make_array());
+    }
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<Datum> DropNullRecordBatch(const std::shared_ptr<RecordBatch>& batch,
+                                  ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (const auto& column : batch->columns()) {
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return batch;
+  }
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch->num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch->num_rows(), true);
+  for (const auto& column : batch->columns()) {
+    if (column->type()->id() == Type::type::NA) {
+      BitUtil::SetBitsTo(dst->mutable_data(), 0, batch->num_rows(), false);
+      break;
+    }
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch->num_rows(), dst, nullptr, 0, 0);
+  if (drop_null_filter->null_count() == batch->num_rows()) {

Review comment:
       This condition is never satisfied, will fix.




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] github-actions[bot] removed a comment on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

Posted by GitBox <gi...@apache.org>.
github-actions[bot] removed a comment on pull request #10802:
URL: https://github.com/apache/arrow/pull/10802#issuecomment-886803563


   <!--
     Licensed to the Apache Software Foundation (ASF) under one
     or more contributor license agreements.  See the NOTICE file
     distributed with this work for additional information
     regarding copyright ownership.  The ASF licenses this file
     to you under the Apache License, Version 2.0 (the
     "License"); you may not use this file except in compliance
     with the License.  You may obtain a copy of the License at
   
       http://www.apache.org/licenses/LICENSE-2.0
   
     Unless required by applicable law or agreed to in writing,
     software distributed under the License is distributed on an
     "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
     KIND, either express or implied.  See the License for the
     specific language governing permissions and limitations
     under the License.
   -->
   
   Thanks for opening a pull request!
   
   If this is not a [minor PR](https://github.com/apache/arrow/blob/master/CONTRIBUTING.md#Minor-Fixes). Could you open an issue for this pull request on JIRA? https://issues.apache.org/jira/browse/ARROW
   
   Opening JIRAs ahead of time contributes to the [Openness](http://theapacheway.com/open/#:~:text=Openness%20allows%20new%20users%20the,must%20happen%20in%20the%20open.) of the Apache Arrow project.
   
   Then could you also rename pull request title in the following format?
   
       ARROW-${JIRA_ID}: [${COMPONENT}] ${SUMMARY}
   
   or
   
       MINOR: [${COMPONENT}] ${SUMMARY}
   
   See also:
   
     * [Other pull requests](https://github.com/apache/arrow/pulls/)
     * [Contribution Guidelines - How to contribute patches](https://arrow.apache.org/docs/developers/contributing.html#how-to-contribute-patches)
   


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,139 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<Array>& column, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  for (int64_t i = 0; i < column->length(); i++) {
+    if (column->IsValid(i)) {
+      builder.Append(static_cast<int32_t>(i));
+    }
+  }
+  RETURN_NOT_OK(builder.Finish(&indices));
+  return indices;
+}
+
+Result<std::shared_ptr<arrow::Array>> GetNotNullIndices(
+    const std::shared_ptr<ChunkedArray>& chunks, MemoryPool* memory_pool) {
+  std::shared_ptr<arrow::Array> indices;
+  arrow::NumericBuilder<arrow::Int32Type> builder(memory_pool);
+  int64_t relative_index = 0;
+  for (int64_t chunk_index = 0; chunk_index < chunks->num_chunks(); ++chunk_index) {
+    auto column_chunk = chunks->chunk(chunk_index);

Review comment:
       Unfortunately, MSVC is opinionated about implicit integral conversions:
   
   https://ci.appveyor.com/project/ApacheSoftwareFoundation/arrow/builds/40193996/job/q88mm6wio371o3nc#L1016
   
   ```suggestion
       auto column_chunk = chunks->chunk(static_cast<int>(chunk_index));
   ```
   
   There are a few similar ~~warning~~errors here which will need to be addressed before CI will be entirely green




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }

Review comment:
       Yes It cant be done avoiding copy +1  




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}

Review comment:
       Yeap, I will create an issue on Jira




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1735,551 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+struct TestDropNullKernelTyped : public TestDropNullKernel {
+  TestDropNullKernelTyped() : rng_(seed_) {}
+
+  template <typename OffsetType>
+  std::vector<OffsetType> Offsets(int32_t length, int32_t slice_count) {
+    std::vector<OffsetType> offsets(static_cast<std::size_t>(slice_count + 1));
+    std::default_random_engine gen(seed_);
+    std::uniform_int_distribution<OffsetType> dist(0, length);
+    std::generate(offsets.begin(), offsets.end(), [&] { return dist(gen); });
+    std::sort(offsets.begin(), offsets.end());
+    return offsets;
+  }

Review comment:
       I change this and you are right, I didn't see that the default null_probability was zero 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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


   I updated this PR base on last feedback comments. Main change, the way algorithm drop nulls now are based on filter operation and  boolean filter array is obtained directly using the bitmap from input array. 


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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }
+      bitmap_offset += bitmap.length();
+    }
+    ::arrow::internal::BitmapAnd(concatenated_bitmap->data(), 0, dst->data(), 0,
+                                 table.num_rows(), 0, dst->mutable_data());
+  }

Review comment:
       Nice! Thanks Niranda




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] bkietz commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection_test.cc
##########
@@ -1734,5 +1734,372 @@ TEST(TestTake, RandomFixedSizeBinary) {
   TakeRandomTest<FixedSizeBinaryType>::Test(fixed_size_binary(16));
 }
 
+// ----------------------------------------------------------------------
+// DropNull tests
+
+void AssertDropNullArrays(const std::shared_ptr<Array>& values,
+                          const std::shared_ptr<Array>& expected) {
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<Array> actual, DropNull(*values));
+  ValidateOutput(actual);
+  AssertArraysEqual(*expected, *actual, /*verbose=*/true);
+}
+
+Status DropNullJSON(const std::shared_ptr<DataType>& type, const std::string& values,
+                    std::shared_ptr<Array>* out) {
+  return DropNull(*ArrayFromJSON(type, values)).Value(out);
+}
+
+void CheckDropNull(const std::shared_ptr<DataType>& type, const std::string& values,
+                   const std::string& expected) {
+  std::shared_ptr<Array> actual;
+
+  ASSERT_OK(DropNullJSON(type, values, &actual));
+  ValidateOutput(actual);
+  AssertArraysEqual(*ArrayFromJSON(type, expected), *actual, /*verbose=*/true);
+}
+
+struct TestDropNullKernel : public ::testing::Test {
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<Array>& values) {
+    ASSERT_EQ(values->null_count(), 0);
+    auto expected = (*DropNull(values)).make_array();
+
+    auto new_values = MakeArray(values->data()->Copy());
+    new_values->data()->buffers[0].reset();
+    new_values->data()->null_count = kUnknownNullCount;
+    auto result = (*DropNull(new_values)).make_array();
+    AssertArraysEqual(*expected, *result);
+  }
+
+  void TestNoValidityBitmapButUnknownNullCount(const std::shared_ptr<DataType>& type,
+                                               const std::string& values) {
+    TestNoValidityBitmapButUnknownNullCount(ArrayFromJSON(type, values));
+  }
+};
+
+TEST_F(TestDropNullKernel, DropNull) {
+  CheckDropNull(null(), "[null, null, null]", "[]");
+  CheckDropNull(null(), "[null]", "[]");
+}
+
+TEST_F(TestDropNullKernel, DropNullBoolean) {
+  CheckDropNull(boolean(), "[true, false, true]", "[true, false, true]");
+  CheckDropNull(boolean(), "[null, false, true]", "[false, true]");
+
+  TestNoValidityBitmapButUnknownNullCount(boolean(), "[true, false, true]");
+}
+
+template <typename ArrowType>
+class TestDropNullKernelTyped : public TestDropNullKernel {};
+
+template <typename ArrowType>
+class TestDropNullKernelWithNumeric : public TestDropNullKernelTyped<ArrowType> {
+ protected:
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(type_singleton(), values, expected);
+  }
+
+  std::shared_ptr<DataType> type_singleton() {
+    return TypeTraits<ArrowType>::type_singleton();
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithNumeric, NumericArrowTypes);
+TYPED_TEST(TestDropNullKernelWithNumeric, DropNullNumeric) {
+  this->AssertDropNull("[7, 8, 9]", "[7, 8, 9]");
+  this->AssertDropNull("[null, 8, 9]", "[8, 9]");
+  this->AssertDropNull("[null, null, null]", "[]");
+}
+
+template <typename TypeClass>
+class TestDropNullKernelWithString : public TestDropNullKernelTyped<TypeClass> {
+ public:
+  std::shared_ptr<DataType> value_type() {
+    return TypeTraits<TypeClass>::type_singleton();
+  }
+
+  void AssertDropNull(const std::string& values, const std::string& expected) {
+    CheckDropNull(value_type(), values, expected);
+  }
+
+  void AssertDropNullDictionary(const std::string& dictionary_values,
+                                const std::string& dictionary_indices,
+                                const std::string& expected_indices) {
+    auto dict = ArrayFromJSON(value_type(), dictionary_values);
+    auto type = dictionary(int8(), value_type());
+    ASSERT_OK_AND_ASSIGN(auto values,
+                         DictionaryArray::FromArrays(
+                             type, ArrayFromJSON(int8(), dictionary_indices), dict));
+    ASSERT_OK_AND_ASSIGN(
+        auto expected,
+        DictionaryArray::FromArrays(type, ArrayFromJSON(int8(), expected_indices), dict));
+    AssertDropNullArrays(values, expected);
+  }
+};
+
+TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryTypes);

Review comment:
       https://ci.appveyor.com/project/ApacheSoftwareFoundation/arrow/builds/40309444/job/ccnm03ackeryl0m8#L1119
   
   ```suggestion
   TYPED_TEST_SUITE(TestDropNullKernelWithString, BinaryArrowTypes);
   ```
   
   These were renamed in #10862 




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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



[GitHub] [arrow] aocsa commented on a change in pull request #10802: ARROW-1568: [C++] Implement Drop Null Kernel for Arrays

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



##########
File path: cpp/src/arrow/compute/kernels/vector_selection.cc
##########
@@ -2146,6 +2147,203 @@ class TakeMetaFunction : public MetaFunction {
   }
 };
 
+// ----------------------------------------------------------------------
+// DropNull Implementation
+
+Status GetDropNullFilter(const Array& values, MemoryPool* memory_pool,
+                         std::shared_ptr<arrow::BooleanArray>* out_array) {
+  auto bitmap_buffer = values.null_bitmap();
+  *out_array = std::make_shared<BooleanArray>(values.length(), bitmap_buffer, nullptr, 0,
+                                              values.offset());
+  return Status::OK();
+}
+
+Status CreateEmptyArray(std::shared_ptr<DataType> type, MemoryPool* memory_pool,
+                        std::shared_ptr<Array>* output_array) {
+  std::unique_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(memory_pool, type, &builder));
+  RETURN_NOT_OK(builder->Resize(0));
+  ARROW_ASSIGN_OR_RAISE(*output_array, builder->Finish());
+  return Status::OK();
+}
+
+Result<std::shared_ptr<Array>> DropNullArray(const std::shared_ptr<Array>& values,
+                                             ExecContext* ctx) {
+  if (values->null_count() == 0) {
+    return values;
+  }
+  if (values->type()->Equals(arrow::null())) {
+    return std::make_shared<NullArray>(0);
+  }
+  std::shared_ptr<BooleanArray> drop_null_filter;
+  RETURN_NOT_OK(GetDropNullFilter(*values, ctx->memory_pool(), &drop_null_filter));
+
+  if (drop_null_filter->null_count() == drop_null_filter->length()) {
+    std::shared_ptr<Array> empty_array;
+    RETURN_NOT_OK(CreateEmptyArray(values->type(), ctx->memory_pool(), &empty_array));
+    return empty_array;
+  }
+  auto options = FilterOptions::Defaults();
+  ARROW_ASSIGN_OR_RAISE(
+      Datum result,
+      CallFunction("array_filter", {Datum(*values), Datum(*drop_null_filter)}, &options,
+                   ctx));
+  return result.make_array();
+}
+
+Result<std::shared_ptr<ChunkedArray>> DropNullChunkedArray(const ChunkedArray& values,
+                                                           ExecContext* ctx) {
+  auto num_chunks = values.num_chunks();
+  std::vector<std::shared_ptr<Array>> new_chunks(num_chunks);
+  for (int i = 0; i < num_chunks; i++) {
+    ARROW_ASSIGN_OR_RAISE(new_chunks[i], DropNullArray(values.chunk(i), ctx));
+  }
+  return std::make_shared<ChunkedArray>(std::move(new_chunks));
+}
+
+Result<std::shared_ptr<RecordBatch>> DropNullRecordBatch(const RecordBatch& batch,
+                                                         ExecContext* ctx) {
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    null_count += column->null_count();
+  }
+  if (null_count == 0) {
+    return RecordBatch::Make(batch.schema(), batch.num_rows(), batch.columns());
+  }
+  if (null_count / batch.num_columns() == batch.num_rows()) {
+    std::vector<std::shared_ptr<Array>> empty_batch(batch.num_columns());
+    for (int i = 0; i < batch.num_columns(); i++) {
+      RETURN_NOT_OK(
+          CreateEmptyArray(batch.column(i)->type(), ctx->memory_pool(), &empty_batch[i]));
+    }
+    return RecordBatch::Make(batch.schema(), 0, empty_batch);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(batch.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, batch.num_rows(), true);
+
+  for (int col_index = 0; col_index < batch.num_columns(); ++col_index) {
+    const auto& column = batch.column(col_index);
+    if (column->null_bitmap_data()) {
+      ::arrow::internal::BitmapAnd(column->null_bitmap_data(), column->offset(),
+                                   dst->data(), 0, column->length(), 0,
+                                   dst->mutable_data());
+    }
+  }
+  auto drop_null_filter =
+      std::make_shared<BooleanArray>(batch.num_rows(), dst, nullptr, 0, 0);
+  ARROW_ASSIGN_OR_RAISE(Datum result, Filter(Datum(batch), Datum(drop_null_filter),
+                                             FilterOptions::Defaults(), ctx));
+  return result.record_batch();
+}
+
+using ::arrow::internal::Bitmap;
+
+Result<std::shared_ptr<Table>> DropNullTable(const Table& table, ExecContext* ctx) {
+  if (table.num_rows() == 0) {
+    return Table::Make(table.schema(), table.columns(), 0);
+  }
+  const int num_columns = table.num_columns();
+  int64_t null_count = 0;
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      null_count += column_chunk->null_count();
+    }
+  }
+  if (null_count == 0) {
+    return Table::Make(table.schema(), table.columns(), table.num_rows());
+  }
+  if (null_count / table.num_columns() == table.num_rows()) {
+    std::vector<std::shared_ptr<ChunkedArray>> empty_table(table.num_columns());
+    for (int i = 0; i < table.num_columns(); i++) {
+      std::shared_ptr<Array> empty_array;
+      RETURN_NOT_OK(
+          CreateEmptyArray(table.column(i)->type(), ctx->memory_pool(), &empty_array));
+      empty_table[i] = std::make_shared<ChunkedArray>(ArrayVector{empty_array});
+    }
+    return Table::Make(table.schema(), empty_table, 0);
+  }
+
+  ARROW_ASSIGN_OR_RAISE(auto dst,
+                        AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+  BitUtil::SetBitsTo(dst->mutable_data(), 0, table.num_rows(), true);
+
+  for (int col_index = 0; col_index < num_columns; ++col_index) {
+    const ArrayVector& chunks = table.column(col_index)->chunks();
+    std::vector<Bitmap> bitmaps(chunks.size());
+    for (size_t chunk_index = 0; chunk_index < chunks.size(); ++chunk_index) {
+      const auto& column_chunk = chunks[chunk_index];
+      bitmaps[chunk_index] = Bitmap(column_chunk->null_bitmap_data(),
+                                    column_chunk->offset(), column_chunk->length());
+    }
+    int64_t bitmap_offset = 0;
+    ARROW_ASSIGN_OR_RAISE(auto concatenated_bitmap,
+                          AllocateEmptyBitmap(table.num_rows(), ctx->memory_pool()));
+    BitUtil::SetBitsTo(concatenated_bitmap->mutable_data(), 0, table.num_rows(), true);
+
+    for (auto bitmap : bitmaps) {
+      if (bitmap.buffer()->data()) {
+        ::arrow::internal::CopyBitmap(bitmap.buffer()->data(), bitmap.offset(),
+                                      bitmap.length(),
+                                      concatenated_bitmap->mutable_data(), bitmap_offset);
+      }

Review comment:
       Yes It cant be done avoiding copy +1  




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

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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