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 2022/01/16 14:17:43 UTC

[GitHub] [arrow] dhruv9vats opened a new pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

dhruv9vats opened a new pull request #12162:
URL: https://github.com/apache/arrow/pull/12162


   Implement a Scalar Kernel to lookup a value for a given key in a `MapArray`, whose type is an alias for `List( Struct(<key>, <item>) )`


-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       Is this even remotely close to what you had in mind when you said refactor @lidavidm ?
   Also, is there a rough example I could follow for `VisitArrayValuesInline` and `UnboxScalar` you mentioned for templating the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,471 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapLookupWithDifferentOptions(const std::shared_ptr<Array>& map,
+                                        const std::shared_ptr<Scalar>& query_key,
+                                        const std::shared_ptr<Array>& expected_all,
+                                        const std::shared_ptr<Array>& expected_first,
+                                        const std::shared_ptr<Array>& expected_last) {
+  MapLookupOptions all_matches(query_key, MapLookupOptions::ALL);
+  MapLookupOptions first_matches(query_key, MapLookupOptions::FIRST);
+  MapLookupOptions last_matches(query_key, MapLookupOptions::LAST);
+
+  CheckScalar("map_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                     expected_first, expected_last);
+}

Review comment:
       Should these be removed now? @lidavidm 




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -254,6 +254,26 @@ struct EnumTraits<compute::RandomOptions::Initializer>
   }
 };
 
+template <>
+struct EnumTraits<compute::MapArrayLookupOptions::Occurrence>
+    : BasicEnumTraits<compute::MapArrayLookupOptions::Occurrence,
+                      compute::MapArrayLookupOptions::Occurrence::FIRST,
+                      compute::MapArrayLookupOptions::Occurrence::LAST,
+                      compute::MapArrayLookupOptions::Occurrence::ALL> {
+  static std::string name() { return "MapArrayLookupOptions::Occurrence"; }
+  static std::string value_name(compute::MapArrayLookupOptions::Occurrence value) {
+    switch (value) {
+      case compute::MapArrayLookupOptions::Occurrence::FIRST:
+        return "FIRST";
+      case compute::MapArrayLookupOptions::Occurrence::LAST:
+        return "LAST";
+      case compute::MapArrayLookupOptions::Occurrence::ALL:
+        return "All";

Review comment:
       ```suggestion
           return "ALL";
   ```




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]
+)";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalarNonRecursive(
+      "map_array_lookup", {map_array},
+      ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookup) {

Review comment:
       But this one fails,
   
   <details><summary> with output </summary>
   
   ```bash
   > GTEST_FILTER="TestScalarNested.MapArrayLookup*" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup*
   25: [==========] Running 2 tests from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 2 tests from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookupNonRecursive
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: [       OK ] TestScalarNested.MapArrayLookupNonRecursive (31 ms)
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:593: [
   25:   "foo",
   25:   "bar",
   25:   "hello",
   25:   "foo",
   25:   "lesgo",
   25:   "whatnow"
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:594: [
   25:   99,
   25:   1,
   25:   2,
   25:   3,
   25:   5,
   25:   8
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:595: Input: -- is_valid: all not null
   25: -- child 0 type: string
   25:   [
   25:     "foo",
   25:     "bar",
   25:     "hello",
   25:     "foo",
   25:     "lesgo",
   25:     "whatnow"
   25:   ]
   25: -- child 1 type: int32
   25:   [
   25:     99,
   25:     1,
   25:     2,
   25:     3,
   25:     5,
   25:     8
   25:   ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:649: MapArrayLookup Scalar result: [
   25:   99,
   25:   3
   25: ] Type: list<item: int32>
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: unknown file: Failure
   25: C++ exception with description "std::bad_cast" thrown in the test body.
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (3 ms)
   25: [----------] 2 tests from TestScalarNested (35 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 2 tests from 1 test suite ran. (35 ms total)
   25: [  PASSED  ] 1 test.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.13 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.13 sec*proc (1 test)
   unittest         =   0.13 sec*proc (1 test)
   
   Total Test time (real) =   0.13 sec
   
   The following tests FAILED:
   	 25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   
   Am getting this same error as above for every option (`FIRST`, `LAST`, `ALL`):
   ```bash
   unknown file: Failure
   C++ exception with description "std::bad_cast" thrown in the test body.
   ```
   and am guessing I'm doing something wrong in `ResolveMapArrayLookupType`




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       Is this even remotely close to what you had in mind @lidavidm ?
   Also, is there a rough example I could follow for `VisitArrayValuesInline` and `UnboxScalar` you mentioned for templating the kernel?

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);

Review comment:
       Note how we no longer have to deal with `offsets()` as before. This does seem to return only the required parts of the `Array`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));
+      }
+    }
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }

Review comment:
       Have kept the null checking outside the helper function as dealing with return types seemed easier this way.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));

Review comment:
       What is the correct approach/way to do this? Especially the `static_pointer_cast` part.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -470,6 +470,30 @@ class ARROW_EXPORT RandomOptions : public FunctionOptions {
   uint64_t seed;
 };
 
+/// Options for map_array_lookup function
+class ARROW_EXPORT MapArrayLookupOptions : public FunctionOptions {
+ public:
+  enum Occurence {
+    /// Return the first matching value
+    First,
+    /// Return the last matching value
+    Last,
+    /// Return all matching values
+    All
+  };
+
+  MapArrayLookupOptions(std::shared_ptr<Scalar> query_key, Occurence occurence = All);
+  MapArrayLookupOptions();
+
+  constexpr static char const kTypeName[] = "MapArrayLookupOptions";
+
+  /// The key to lookup in the map
+  std::shared_ptr<Scalar> query_key;

Review comment:
       If `query_key` should be a comparable type, what should be its `DataType/Shape` 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));

Review comment:
       Had to use the `AppendScalar` approach, as using
   ```cpp
   RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
   ```
   caused problems for me. (Is this even the correct way of using `AppendArraySlice`?)
   <details><summary>And for the current tests, AppendArraySlice failed with: </summary>
   
   ```bash
   ❯ GTEST_FILTER="TestScalarNested.MapArrayLookup" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup
   25: [==========] Running 1 test from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 1 test from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Key being checked: foo
   25: Item at key: 101
   25: Key being checked: foo
   25: Item at key: 22
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -99
   25: -3
   25: -101
   25: -22
   25: +65635
   25: +0
   25: +0
   25: +0
   25: Expected:
   25:   [
   25:     99,
   25:     3,
   25:     101,
   25:     22
   25:   ]
   25: Actual:
   25:   [
   25:     65635,
   25:     0,
   25:     0,
   25:     0
   25:   ]
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -99
   25: +65635
   25: Expected:
   25:   [
   25:     99
   25:   ]
   25: Actual:
   25:   [
   25:     65635
   25:   ]
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (8 ms)
   25: [----------] 1 test from TestScalarNested (8 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 1 test from 1 test suite ran. (8 ms total)
   25: [  PASSED  ] 0 tests.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.10 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.10 sec*proc (1 test)
   unittest         =   0.10 sec*proc (1 test)
   
   Total Test time (real) =   0.10 sec
   
   The following tests FAILED:
            25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,167 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookupNested) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto foo_all_output = ArrayFromJSON(
+      list(map(int16(), int16())),
+      "[ [[[4, 4], [5, 5]], [[8, 8], [9, 9]], [[12, 12], [13, 13]]], null, [[[4, 5], [5, "
+      "6]], [[8, 9], [9, 10]], [[12, 13], [13, 14]]], null ]");
+
+  CheckScalar("map_array_lookup", {map_array}, foo_all_output, &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[4, 4], [5, 5]], null, [[4, 5], [5, 6]], null ]"),
+              &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[12, 12], [13, 13]], null, [[12, 13], [13, 14]], null ]"),
+              &foo_last);
+}
+
+template <typename Type>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, IntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {
+  auto type = default_type_instance<TypeParam>();
+
+  auto one_scalar = MakeScalar(type, 1).ValueOrDie();
+  MapArrayLookupOptions one_all(one_scalar, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions one_first(one_scalar, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions one_last(one_scalar, MapArrayLookupOptions::LAST);
+
+  auto map_type = map(type, utf8());
+  const char* input = R"(
+    [
+      [ 
+        [0, "zero"], [1, "first_one"], [2, "two"], [3, "three"], [1, "second_one"],
+        [1, "last_one"]
+      ],
+      null,
+      [ 
+        [0, "zero_hero"], [9, "almost_six"], [1, "the_dumb_one"], [7, "eleven"],
+        [1, "the_chosen_one"], [42, "meaning of life?"], [1, "just_one"],
+        [1, "no more ones!"]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(map_type, input);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["first_one", null, "the_dumb_one", null])"),
+              &one_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["last_one", null, "no more ones!", null])"),
+              &one_last);
+  CheckScalar("map_array_lookup", {map_array}, ArrayFromJSON(list(utf8()), R"([

Review comment:
       For the last case, use TweakValidityBit: https://github.com/apache/arrow/blob/c39caedc2e9296fcb53d1c587708836ec6628ddb/cpp/src/arrow/testing/gtest_util.h#L347-L353
   
   ```cpp
   auto arr = ArrayFromJSON(ty, "[["foo", 1]]");
   auto tweaked = TweakValidityBit(arr, 0, false); // Now arr[0] is null, but the underlying data is still there
   ```




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([

Review comment:
       Okay, sorry, my bad.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());

Review comment:
       Doing something like:
   ```cpp
   auto descrs = batch.GetDescriptors();
   std::shared_ptr<DataType> type = descrs.front().type;
   std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
   ```
   
   Like in `ResolveMapArrayLookupType` okay?




-- 
This is an automated message from the 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] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;

Review comment:
       The idea was, `list_struct_len` will be 0, when a map_array element is not valid, so the inner loop will be skipped either way.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,167 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookupNested) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto foo_all_output = ArrayFromJSON(
+      list(map(int16(), int16())),
+      "[ [[[4, 4], [5, 5]], [[8, 8], [9, 9]], [[12, 12], [13, 13]]], null, [[[4, 5], [5, "
+      "6]], [[8, 9], [9, 10]], [[12, 13], [13, 14]]], null ]");
+
+  CheckScalar("map_array_lookup", {map_array}, foo_all_output, &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[4, 4], [5, 5]], null, [[4, 5], [5, 6]], null ]"),
+              &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[12, 12], [13, 13]], null, [[12, 13], [13, 14]], null ]"),
+              &foo_last);
+}
+
+template <typename Type>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, IntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {
+  auto type = default_type_instance<TypeParam>();
+
+  auto one_scalar = MakeScalar(type, 1).ValueOrDie();
+  MapArrayLookupOptions one_all(one_scalar, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions one_first(one_scalar, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions one_last(one_scalar, MapArrayLookupOptions::LAST);
+
+  auto map_type = map(type, utf8());
+  const char* input = R"(
+    [
+      [ 
+        [0, "zero"], [1, "first_one"], [2, "two"], [3, "three"], [1, "second_one"],
+        [1, "last_one"]
+      ],
+      null,
+      [ 
+        [0, "zero_hero"], [9, "almost_six"], [1, "the_dumb_one"], [7, "eleven"],
+        [1, "the_chosen_one"], [42, "meaning of life?"], [1, "just_one"],
+        [1, "no more ones!"]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(map_type, input);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["first_one", null, "the_dumb_one", null])"),
+              &one_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["last_one", null, "no more ones!", null])"),
+              &one_last);
+  CheckScalar("map_array_lookup", {map_array}, ArrayFromJSON(list(utf8()), R"([

Review comment:
       Also, I've only been writing simple `CheckScalar` form tests till now. What should be the things I should also test for here? @lidavidm 




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);

Review comment:
       The length of the current `List`, aka, the number of `keys` to check in the `keys` `arrow::Array` above (which is _continous_ in memory and are separated by offsets).
   Have I understood this correctly, and is using the `map_array.value_length(map_array_idx)`, the correct way to obtain it?




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));

Review comment:
       Had to cast because for some reason, for the current tests, the `item` obtained above had the type `int16`, while the `item_type` was `int32`, which is also the `DataType` of the `Array` being built. 
   It gave the error:
   ```text
   failed with Invalid: Cannot append scalar of type int16 to builder for type int32
   ```




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;

Review comment:
       To iterate through the `List`, where each element is a `Struct( <key: item>)`.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {
+    std::cout << "map type found!\n";
+    key_type = type->field(0)->type()->field(0)->type();

Review comment:
       Is this a good place to check whether the `query_key` type and `Map::key` type match?




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -573,6 +605,7 @@ void RegisterScalarOptions(FunctionRegistry* registry) {
   DCHECK_OK(registry->AddFunctionOptionsType(kUtf8NormalizeOptionsType));
   DCHECK_OK(registry->AddFunctionOptionsType(kWeekOptionsType));
   DCHECK_OK(registry->AddFunctionOptionsType(kRandomOptionsType));
+  DCHECK_OK(registry->AddFunctionOptionsType(kMapArrayLookupOptionsType));

Review comment:
       ditto here

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -344,6 +364,9 @@ static auto kRandomOptionsType = GetFunctionOptionsType<RandomOptions>(
     DataMember("length", &RandomOptions::length),
     DataMember("initializer", &RandomOptions::initializer),
     DataMember("seed", &RandomOptions::seed));
+static auto kMapArrayLookupOptionsType = GetFunctionOptionsType<MapArrayLookupOptions>(

Review comment:
       nit, but try to keep these organized alphabetically (though I see Random already broke that)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {

Review comment:
       No need for this check. We're only registering kernels that accept MapType parameters so this should never _not_ be MAP. (You can DCHECK_EQ if you really want to be sure.)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, 3, 101, 22]"), &foo_all);

Review comment:
       This output seems wrong, I would expect `[[99, 3], null, [101, 22], null]`

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {
+    std::cout << "map type found!\n";
+    key_type = type->field(0)->type()->field(0)->type();
+    value_type = type->field(0)->type()->field(1)->type();
+
+    std::cout << "Value type: " << value_type->ToString() << "\n";
+  }
+  return ValueDescr(value_type, descrs.front().shape);

Review comment:
       Eventually if Occurrence == ALL then this should return `ValueDescr(list(value_type), descrs.front().shape)`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {

Review comment:
       There's no one "correct" structure. This is probably okay.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -545,6 +568,15 @@ RandomOptions::RandomOptions(int64_t length, Initializer initializer, uint64_t s
 RandomOptions::RandomOptions() : RandomOptions(0, SystemRandom, 0) {}
 constexpr char RandomOptions::kTypeName[];
 
+MapArrayLookupOptions::MapArrayLookupOptions(std::shared_ptr<Scalar> query_key,
+                                             Occurence occurence)
+    : FunctionOptions(internal::kMapArrayLookupOptionsType),
+      query_key(query_key),

Review comment:
       nit, but `std::move(query_key)`

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -470,6 +470,30 @@ class ARROW_EXPORT RandomOptions : public FunctionOptions {
   uint64_t seed;
 };
 
+/// Options for map_array_lookup function
+class ARROW_EXPORT MapArrayLookupOptions : public FunctionOptions {
+ public:
+  enum Occurence {
+    /// Return the first matching value
+    First,

Review comment:
       For these enums we use UPPER_SNAKE_CASE naming.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -545,6 +568,15 @@ RandomOptions::RandomOptions(int64_t length, Initializer initializer, uint64_t s
 RandomOptions::RandomOptions() : RandomOptions(0, SystemRandom, 0) {}
 constexpr char RandomOptions::kTypeName[];
 
+MapArrayLookupOptions::MapArrayLookupOptions(std::shared_ptr<Scalar> query_key,

Review comment:
       ditto here

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];

Review comment:
       `ArrayFromJSON` directly supports MapType, use that. For instance https://github.com/apache/arrow/blob/e12a4545bdc5a8683c8dfdbb0468922d444c0500/cpp/src/arrow/ipc/json_simple_test.cc#L757

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([

Review comment:
       This is why you're seeing int16 in your implementation above.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, 3, 101, 22]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array}, ArrayFromJSON(int32(), "[99]"),

Review comment:
       A scalar kernel is elementwise. There should be exactly one output element per input element. The output here should be `[99, null, 101, null]`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);

Review comment:
       This is not validating the types/values, hence the Cast above. Use ArrayFromJSON.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {
+    std::cout << "map type found!\n";
+    key_type = type->field(0)->type()->field(0)->type();
+    value_type = type->field(0)->type()->field(1)->type();
+
+    std::cout << "Value type: " << value_type->ToString() << "\n";
+  }
+  return ValueDescr(value_type, descrs.front().shape);
+}
+
+void AddMapArrayLookupKernels(ScalarFunction* func) {
+  ScalarKernel kernel(
+      {InputType(Type::MAP, ValueDescr::ARRAY)}, OutputType(ResolveMapArrayLookupType),
+      MapArrayLookupFunctor::ExecMapArray, OptionsWrapper<MapArrayLookupOptions>::Init);
+  kernel.null_handling = NullHandling::COMPUTED_NO_PREALLOCATE;
+  kernel.mem_allocation = MemAllocation::NO_PREALLOCATE;
+  DCHECK_OK(func->AddKernel(std::move(kernel)));
+}
+
+const FunctionDoc map_array_lookup_doc(
+    "Find the items corresponding to a given key in a MapArray", ("More doc"),
+    {"container"}, "MapArrayLookupOptions");

Review comment:
       ```suggestion
       {"container"}, "MapArrayLookupOptions", /*options_required=*/true);
   ```

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -470,6 +470,30 @@ class ARROW_EXPORT RandomOptions : public FunctionOptions {
   uint64_t seed;
 };
 
+/// Options for map_array_lookup function
+class ARROW_EXPORT MapArrayLookupOptions : public FunctionOptions {
+ public:
+  enum Occurence {
+    /// Return the first matching value
+    First,
+    /// Return the last matching value
+    Last,
+    /// Return all matching values
+    All
+  };
+
+  MapArrayLookupOptions(std::shared_ptr<Scalar> query_key, Occurence occurence = All);
+  MapArrayLookupOptions();
+
+  constexpr static char const kTypeName[] = "MapArrayLookupOptions";
+
+  /// The key to lookup in the map
+  std::shared_ptr<Scalar> query_key;

Review comment:
       Shape does not apply to Scalar. (Scalar, Array, etc. are Shapes.)
   
   For `MapType[K, V]`, `query_key` should be of type `K`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {
+    std::cout << "map type found!\n";
+    key_type = type->field(0)->type()->field(0)->type();

Review comment:
       Yes, you can check them here and return an error, you could do it in the kernel as well.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> value_type;
+  std::shared_ptr<DataType> key_type;
+  if (type->id() == Type::MAP) {
+    std::cout << "map type found!\n";
+    key_type = type->field(0)->type()->field(0)->type();

Review comment:
       Instead of using `type->field...` try `checked_cast<const MapType&>(*type).key_type()`

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));

Review comment:
       Look below. You have int16, not int32, in the tests. Use ArrayFromJSON directly instead of trying to build up the array from its parts.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,40 +428,82 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
-
+template <typename KeyType>
 struct MapArrayLookupFunctor {
-  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
                                               const int64_t start, const int64_t end,
                                               const bool from_back = false) {
-    if (!from_back) {
-      for (int64_t idx = start; idx < end; ++idx) {
-        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
-
-        if (key->Equals(query_key)) return idx;
-      }
-    } else {
-      for (int64_t idx = end - 1; idx >= start; --idx) {
-        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              if (!from_back) {
+                match_idx = index;
+                return Status::Cancelled("Found first matching key");
+              } else {
+                match_idx = index;
+              }
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
 
-        if (key->Equals(query_key)) return idx;
-      }
-    }
-    return -1;
+    return match_idx;
   }
 
   static Result<std::unique_ptr<ArrayBuilder>> GetBuiltArray(

Review comment:
       This could perhaps take in `ArrayBuilder* builder` as a parameter instead of returning it. That way, for the Array case below, it could directly append to the child items builder of the list builder, avoiding an extra copy of  the data/allocation of a scalar. And the Scalar case could just use it as normal.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {

Review comment:
       The latter will allocate a new Array which will do some shared_ptr copies. That's not too bad, though, so if it makes the code clearer, let's go with that. (We can adjust if it turns out to be significant overhead but I'm not too sure it will. If so, we probably need to add explicit offset/length support to VisitArrayValuesInline) 

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {

Review comment:
       Hmm, I also see that there's some duplicate logic below for the array case. Instead of having this return a single index, you could organize it as follows:
   
   ```
   template <typename FoundItem>
   Status FindMatchingIndices(const Array& keys, const Scalar& query, const bool from_back, FoundItem callback);
   
   RETURN_NOT_OK(FindMatchingIndices(...field(0), query_key_scalar, from_back, [&](int64_t index) {
     // Scalar case: append to builder or call GetScalar()
     // Array case: append to ItemType builder or List<ItemType> builder
   });
   ```
   
   that way all the logic gets centralized

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              if (!from_back) {
+                match_idx = index;
+                return Status::Cancelled("Found first matching key");
+              } else {
+                match_idx = index;
+              }
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return match_idx;
+  }
+
+  static Result<std::unique_ptr<ArrayBuilder>> GetBuiltArray(
+      const Array& keys, const Array& items, const Scalar& query_key_scalar,
+      bool& found_at_least_one_key, const int64_t& start, const int64_t& end,
+      KernelContext* ctx) {
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return std::move(builder);
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+    auto offsets = std::dynamic_pointer_cast<Int32Array>(map_array.offsets());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        std::unique_ptr<ArrayBuilder> list_builder;
+        bool found_at_least_one_key = false;
+        ARROW_ASSIGN_OR_RAISE(
+            list_builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key,
+                                        start, end, ctx));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+        ARROW_ASSIGN_OR_RAISE(
+            int64_t key_match_idx,
+            FindOneMapValueIndex(*keys, *query_key, start, end, from_back));
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      ARROW_ASSIGN_OR_RAISE(
+          builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key, 0,
+                                 struct_array.length(), ctx));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||
+      !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_array_lookup: query_key type and MapArray key_type don't match. Expected "
+        "type: ",
+        *item_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else { /* occurrence == FIRST || LAST */
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+struct ResolveMapArrayLookup {
+  KernelContext* ctx;
+  const ExecBatch& batch;
+  Datum* out;
+
+  template <typename KeyType>
+  Status Execute() {
+    if (batch[0].kind() == Datum::SCALAR) {
+      return MapArrayLookupFunctor<KeyType>::ExecMapScalar(ctx, batch, out);
+    }
+    return MapArrayLookupFunctor<KeyType>::ExecMapArray(ctx, batch, out);
+  }
+
+  template <typename KeyType>
+  enable_if_physical_integer<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_decimal<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_base_binary<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_boolean<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+

Review comment:
       For MonthDayNanoInterval, you can just overload it like normal (`Status Visit(const MonthDayNanoInterval& type)`). I think `VisitArrayValuesInline` should work (the type should be the `MonthDayNanos` struct: https://github.com/apache/arrow/blob/8daa7a4ed5629c0020dadf7325a6b523bdfc62e9/cpp/src/arrow/type.h#L1382)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              if (!from_back) {
+                match_idx = index;
+                return Status::Cancelled("Found first matching key");
+              } else {
+                match_idx = index;
+              }
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return match_idx;
+  }
+
+  static Result<std::unique_ptr<ArrayBuilder>> GetBuiltArray(
+      const Array& keys, const Array& items, const Scalar& query_key_scalar,
+      bool& found_at_least_one_key, const int64_t& start, const int64_t& end,
+      KernelContext* ctx) {
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return std::move(builder);
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+    auto offsets = std::dynamic_pointer_cast<Int32Array>(map_array.offsets());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        std::unique_ptr<ArrayBuilder> list_builder;
+        bool found_at_least_one_key = false;
+        ARROW_ASSIGN_OR_RAISE(
+            list_builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key,
+                                        start, end, ctx));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+        ARROW_ASSIGN_OR_RAISE(
+            int64_t key_match_idx,
+            FindOneMapValueIndex(*keys, *query_key, start, end, from_back));
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      ARROW_ASSIGN_OR_RAISE(
+          builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key, 0,
+                                 struct_array.length(), ctx));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||
+      !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_array_lookup: query_key type and MapArray key_type don't match. Expected "
+        "type: ",
+        *item_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else { /* occurrence == FIRST || LAST */
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+struct ResolveMapArrayLookup {
+  KernelContext* ctx;
+  const ExecBatch& batch;
+  Datum* out;
+
+  template <typename KeyType>
+  Status Execute() {
+    if (batch[0].kind() == Datum::SCALAR) {
+      return MapArrayLookupFunctor<KeyType>::ExecMapScalar(ctx, batch, out);
+    }
+    return MapArrayLookupFunctor<KeyType>::ExecMapArray(ctx, batch, out);
+  }
+
+  template <typename KeyType>
+  enable_if_physical_integer<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_decimal<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_base_binary<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_boolean<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+

Review comment:
       You can use `enable_if_same` to provide an overload specifically for FixedSizeBinaryType




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,273 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool& from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildListOfItemsArray(const Array& keys, const Array& items,
+                                      const Scalar& query_key_scalar,
+                                      bool& found_at_least_one_key,
+                                      ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          Status to_return = Status::OK();
+          if (key == query_key) {
+            to_return = callback(index);
+          }
+          ++index;
+          return to_return;

Review comment:
       ```suggestion
             if (key == query_key) {
               return callback(index++);
             }
             ++index;
             return Status::OK();
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,423 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+TEST_F(TestMapArrayLookupKernel, BooleanKey) {
+  auto true_scalar = ScalarFromJSON(boolean(), R"(true)");
+  auto map_type = map(boolean(), int32());
+  const char* input = R"(
+    [
+      [
+        [true, 99], [false, 1], [false, 2], [true, null], [false, 5],
+        [true, 8]
+      ],
+      null,
+      [
+        [false, null], [true, 67], [false, 101], [false, 1], [false, null],
+        [false, 9], [true, 80]
+      ],
+      [],
+      [
+        [false, 1], [false, 2], [false, 3], [false, 4]
+      ],
+      [
+        [true, 9], [true, 2], [true, 5], [true, 8]
+      ]
+    ]
+  )";
+  auto map_array = ArrayFromJSON(map_type, input);
+  auto map_array_tweaked = TweakValidityBit(map_array, 5, false);
+
+  auto expected_all = ArrayFromJSON(list(int32()), R"(
+    [[99, null, 8], null, [67, 80], null, null, null ])");
+  auto expected_first = ArrayFromJSON(int32(), "[99, null, 67, null, null, null]");
+  auto expected_last = ArrayFromJSON(int32(), "[8, null, 80, null, null, null]");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array_tweaked, true_scalar, expected_all,
+                                          expected_first, expected_last);
+}
+
+TEST_F(TestMapArrayLookupKernel, MonthDayNanoIntervalKeys) {
+  auto key_type = month_day_nano_interval();
+  auto map_type = map(key_type, utf8());
+  auto key_scalar = ScalarFromJSON(month_day_nano_interval(), R"([1, 2, -3])");
+  const char* input = R"(
+    [
+      [
+        [[-9, -10, 11], "zero"], [[1, 2, -3], "first_one"], [[11, -12, 0], "two"],
+        [[1, 2, -3], null], [[-7, -8, -9], "three"], [[1, 2, -3], "second_one"],
+        [[1, 2, -3], "last_one"]
+      ],
+      null,
+      [
+        [[-5, 6, 7], "zero_hero"], [[15, 16, 2], "almost_six"],
+        [[1, 2, -3], "the_dumb_one"], [[-7, -8, -9], "eleven"],
+        [[1, 2, -3], "the_chosen_one"], [[-5, 6, 7], "meaning of life"],
+        [[1, 2, -3], "just_one"], [[1, 2, -3], "no more ones!"]
+      ],
+      [
+        [[-5, 6, 7], "this"], [[-13, 14, -1], "has"], [[11, -12, 0], "no"],
+        [[15, 16, 2], "keys"]
+      ],
+      [
+        [[1, 2, -3], "this"], [[1, 2, -3], "should"], [[1, 2, -3], "also"],
+        [[1, 2, -3], "be"], [[1, 2, -3], "null"]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(map_type, input);
+  auto map_array_tweaked = TweakValidityBit(map_array, 4, false);
+
+  auto expected_first =
+      ArrayFromJSON(utf8(), R"(["first_one", null, "the_dumb_one", null, null, null])");
+  auto expected_last =
+      ArrayFromJSON(utf8(), R"(["last_one", null, "no more ones!", null, null, null])");
+  auto expected_all = ArrayFromJSON(list(utf8()),
+                                    R"([
+                                          ["first_one", null, "second_one", "last_one"],
+                                          null,
+                                          ["the_dumb_one", "the_chosen_one", "just_one", "no more ones!"],
+                                          null,
+                                          null,
+                                          null
+                                        ]
+                                      )");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array_tweaked, key_scalar, expected_all,
+                                          expected_first, expected_last);
+}

Review comment:
       Eventually we should add a `TEST(TestMapArrayLookupKernel, Errors)` where we check error conditions like not providing a scalar or providing a scalar of the wrong type. (`ASSERT_RAISES(TypeError, MapArrayLookup(...));`)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,423 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+TEST_F(TestMapArrayLookupKernel, BooleanKey) {
+  auto true_scalar = ScalarFromJSON(boolean(), R"(true)");
+  auto map_type = map(boolean(), int32());
+  const char* input = R"(
+    [
+      [
+        [true, 99], [false, 1], [false, 2], [true, null], [false, 5],
+        [true, 8]
+      ],
+      null,
+      [
+        [false, null], [true, 67], [false, 101], [false, 1], [false, null],
+        [false, 9], [true, 80]
+      ],
+      [],
+      [
+        [false, 1], [false, 2], [false, 3], [false, 4]
+      ],
+      [
+        [true, 9], [true, 2], [true, 5], [true, 8]

Review comment:
       In general: can we add some `null` keys in addition to `true` and `false`? (Same below.)

##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1375,20 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Finds either the FIRST, LAST, or ALL items with a key that matches the given
+/// query key in a map array.
+///
+/// Returns an array of items for FIRST and LAST, and an array of list of items for ALL.
+///
+/// \param[in] map_array to look in
+/// \param[in] options to pass a query key and choose which matching keys to return
+/// (FIRST, LAST or ALL)
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> MapArrayLookup(const Datum& map_array,
+                                          MapArrayLookupOptions options,
+                                          ExecContext* ctx = NULLPTR);

Review comment:
       Yes, this is fine.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,273 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool& from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildListOfItemsArray(const Array& keys, const Array& items,
+                                      const Scalar& query_key_scalar,
+                                      bool& found_at_least_one_key,
+                                      ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          Status to_return = Status::OK();
+          if (key == query_key) {
+            to_return = callback(index);
+          }
+          ++index;
+          return to_return;

Review comment:
       mostly a nit, just to slim things a little

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,273 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool& from_back) {

Review comment:
       Sure, this works.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -785,5 +819,12 @@ Result<Datum> Week(const Datum& arg, WeekOptions options, ExecContext* ctx) {
   return CallFunction("week", {arg}, &options, ctx);
 }
 
+// ----------------------------------------------------------------------
+
+Result<Datum> MapArrayLookup(const Datum& arg, MapArrayLookupOptions options,
+                             ExecContext* ctx) {
+  return CallFunction("map_array_lookup", {arg}, &options, ctx);
+}
+

Review comment:
       Here is fine, I think
   
   Add a comment `// Structural transforms` below the `// ---` line to be consistent with the other sections




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -17,6 +17,7 @@
 
 // Vector kernels involving nested types
 
+#include <iostream>

Review comment:
       Added for debugging, will remove later. Sorry for all the print statements that are littered throughout.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);

Review comment:
       Note how we no longer have to deal with `offsets()` as before. This does seem to return only the required parts of the `Array`, which I now recall you initially pointed out in the JIRA. But does this incur some performance penalty?
   What would be the correct way to do 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},

Review comment:
       CheckScalar requires that you implement the Scalar case as well but let's just get this to work first.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {

Review comment:
       This test passes.
   
   <details><summary> with output </summary>
   
   ```bash
   > GTEST_FILTER="TestScalarNested.MapArrayLookupNonRecursive" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookupNonRecursive
   25: [==========] Running 1 test from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 1 test from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookupNonRecursive
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: [       OK ] TestScalarNested.MapArrayLookupNonRecursive (37 ms)
   25: [----------] 1 test from TestScalarNested (37 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 1 test from 1 test suite ran. (37 ms total)
   25: [  PASSED  ] 1 test.
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........   Passed    0.15 sec
   
   The following tests passed:
   	arrow-compute-scalar-test
   
   100% tests passed, 0 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.15 sec*proc (1 test)
   unittest         =   0.15 sec*proc (1 test)
   
   Total Test time (real) =   0.15 sec
   
   ```
   
   </details>




-- 
This is an automated message from the 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] lidavidm commented on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Thanks @dhruv9vats!


-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,471 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapLookupWithDifferentOptions(const std::shared_ptr<Array>& map,
+                                        const std::shared_ptr<Scalar>& query_key,
+                                        const std::shared_ptr<Array>& expected_all,
+                                        const std::shared_ptr<Array>& expected_first,
+                                        const std::shared_ptr<Array>& expected_last) {
+  MapLookupOptions all_matches(query_key, MapLookupOptions::ALL);
+  MapLookupOptions first_matches(query_key, MapLookupOptions::FIRST);
+  MapLookupOptions last_matches(query_key, MapLookupOptions::LAST);
+
+  CheckScalar("map_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                     expected_first, expected_last);
+}

Review comment:
       Yes, I'm not sure these tests add anything compared to the existing ones.




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       - So, this was the basic structure I was thinking of,
   ```cpp
     template <typename KeyType>
     struct MapArrayLookupFunctor {
     .
     .
     .
     const KeyType query_key = UnboxScalar<KeyType>::Unbox(*options.query_key);
     ...
     const std::shared_ptr<Array> keys = struct_array.field(0);
     ...
     
     VisitArrayValuesInline<KeyType>(
       *keys->data(),
       [&](KeyType key) -> Status {
         if (key == query_key) {
           if (occurrence == FIRST) {
             ...
             return Status::Cancelled("Found key");
           } else /*occurrence == ALL*/ { 
               ...
               return Status::OK(); 
           }
         } else { return Status::OK(); }
        },
       [&]() -> Status {
         ...
         return Status::OK();
       }
     );
     ``` 
   - But as the values are visited _inline_, how would we deal with the `LAST` option (iterating from the back)?
   - And just so I'm clear, when you say templating the kernel, you mean templating for the `KeyType`, right?
   

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       So it's basically back to the offsets approach now, right? Which did feel more _elegant_ :)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       But offsets are only needed in the Array case, as MapScalar -> StructArray -> field(0, 1) approach for the Scalar case give the _appropriate_(corresponding just to the given map element from the MapArray) keys/items anyways.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       So it's basically back to the offsets approach now, right? Which did feel more _elegant_ :)




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());

Review comment:
       Ah sorry, the cast from Scalar to MapScalar is fine - it's the cast from MapScalar.value to StructArray that is failing, because `value` is nullptr when `is_valid == false`.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       Sure, but that case can just have offset=0. And I'm suggesting this since this is the code that will change the most when the kernel gets templated, so it will save some effort changing it then.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,423 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+TEST_F(TestMapArrayLookupKernel, BooleanKey) {
+  auto true_scalar = ScalarFromJSON(boolean(), R"(true)");
+  auto map_type = map(boolean(), int32());
+  const char* input = R"(
+    [
+      [
+        [true, 99], [false, 1], [false, 2], [true, null], [false, 5],
+        [true, 8]
+      ],
+      null,
+      [
+        [false, null], [true, 67], [false, 101], [false, 1], [false, null],
+        [false, 9], [true, 80]
+      ],
+      [],
+      [
+        [false, 1], [false, 2], [false, 3], [false, 4]
+      ],
+      [
+        [true, 9], [true, 2], [true, 5], [true, 8]

Review comment:
       This causes error: https://github.com/apache/arrow/blob/a3efe72f99b1b9f23b1d11afc648f4306d32e330/cpp/src/arrow/ipc/json_simple.cc#L585-L586




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;

Review comment:
       Using this in conjunction with the condition above to break out of nested `for` loops if `Occurence::First` is used. As this currently is, this will break out when the first key is encountered, regardless of whether there are matching keys in _other_ `Lists`. Is this what we want?
   
   If instead _all_ `First` keys (and its item) from *each* `List` is needed, removing this condition will (I hope) suffice.
   
   There was also a mention of returning a different `DataType` when `First` is used as opposed to `All`, so how does this all work?
   




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {

Review comment:
       To iterate through `MapArray`, where each element is a `List( Struct( key : value ))`.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, 3, 101, 22]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array}, ArrayFromJSON(int32(), "[99]"),

Review comment:
       Also, this _very_ basic test does (somehow) pass:
   
   <details><summary>Passing test output</summary>
   
   ```bash
   ❯ GTEST_FILTER="TestScalarNested.MapArrayLookup" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup
   25: [==========] Running 1 test from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 1 test from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Value at key: 99
   25: Item being appended: 99
   25: Key being checked: foo
   25: Value at key: 3
   25: Item being appended: 3
   25: Key being checked: foo
   25: Value at key: 101
   25: Item being appended: 101
   25: Key being checked: foo
   25: Value at key: 22
   25: Item being appended: 22
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Value at key: 99
   25: Item being appended: 99
   25: [       OK ] TestScalarNested.MapArrayLookup (10 ms)
   25: [----------] 1 test from TestScalarNested (10 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 1 test from 1 test suite ran. (10 ms total)
   25: [  PASSED  ] 1 test.
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........   Passed    0.08 sec
   
   The following tests passed:
           arrow-compute-scalar-test
   
   100% tests passed, 0 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.08 sec*proc (1 test)
   unittest         =   0.08 sec*proc (1 test)
   
   Total Test time (real) =   0.08 sec
   ```
   
   </details>
   
   (Please do suggest better tests)




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

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

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



[GitHub] [arrow] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Failed :arrow_down:0.36% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Finished :arrow_down:0.48% :arrow_up:0.78%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] lidavidm commented on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Also just note a few lints from CI:
   
   ```
   pyarrow.compute.map_array_lookup
   -> pyarrow.compute.Find the items corresponding to a given key in a MapArray.
   PR01: Parameters {'scalar'} not documented
   
   pyarrow._compute.MapArrayLookupOptions
   PR01: Parameters {'scalar'} not documented
   ```


-- 
This is an automated message from the 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] ursabot commented on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Scheduled] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,167 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookupNested) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto foo_all_output = ArrayFromJSON(
+      list(map(int16(), int16())),
+      "[ [[[4, 4], [5, 5]], [[8, 8], [9, 9]], [[12, 12], [13, 13]]], null, [[[4, 5], [5, "
+      "6]], [[8, 9], [9, 10]], [[12, 13], [13, 14]]], null ]");
+
+  CheckScalar("map_array_lookup", {map_array}, foo_all_output, &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[4, 4], [5, 5]], null, [[4, 5], [5, 6]], null ]"),
+              &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[12, 12], [13, 13]], null, [[12, 13], [13, 14]], null ]"),
+              &foo_last);
+}
+
+template <typename Type>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, IntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {

Review comment:
       I would say these are the "groups" you need: `PhysicalIntegralArrowTypes`, `DecimalArrowTypes`, `BaseBinaryArrowTypes`, and then miscellaneous types (Boolean, FixedSizeString, MonthDayNanoInterval)
   
   See here for all of the categories (note they overlap/not all are relevant): https://github.com/apache/arrow/blob/c39caedc2e9296fcb53d1c587708836ec6628ddb/cpp/src/arrow/testing/gtest_util.h#L153-L185

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,167 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookupNested) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto foo_all_output = ArrayFromJSON(
+      list(map(int16(), int16())),
+      "[ [[[4, 4], [5, 5]], [[8, 8], [9, 9]], [[12, 12], [13, 13]]], null, [[[4, 5], [5, "
+      "6]], [[8, 9], [9, 10]], [[12, 13], [13, 14]]], null ]");
+
+  CheckScalar("map_array_lookup", {map_array}, foo_all_output, &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[4, 4], [5, 5]], null, [[4, 5], [5, 6]], null ]"),
+              &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[12, 12], [13, 13]], null, [[12, 13], [13, 14]], null ]"),
+              &foo_last);
+}
+
+template <typename Type>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, IntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {
+  auto type = default_type_instance<TypeParam>();
+
+  auto one_scalar = MakeScalar(type, 1).ValueOrDie();
+  MapArrayLookupOptions one_all(one_scalar, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions one_first(one_scalar, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions one_last(one_scalar, MapArrayLookupOptions::LAST);
+
+  auto map_type = map(type, utf8());
+  const char* input = R"(
+    [
+      [ 
+        [0, "zero"], [1, "first_one"], [2, "two"], [3, "three"], [1, "second_one"],
+        [1, "last_one"]
+      ],
+      null,
+      [ 
+        [0, "zero_hero"], [9, "almost_six"], [1, "the_dumb_one"], [7, "eleven"],
+        [1, "the_chosen_one"], [42, "meaning of life?"], [1, "just_one"],
+        [1, "no more ones!"]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(map_type, input);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["first_one", null, "the_dumb_one", null])"),
+              &one_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(utf8(), R"(["last_one", null, "no more ones!", null])"),
+              &one_last);
+  CheckScalar("map_array_lookup", {map_array}, ArrayFromJSON(list(utf8()), R"([

Review comment:
       Hmm, some cases to consider: empty maps, null maps (both already covered); null keys, null values (do we want to support looking up null keys? should consider that), and null maps where the values are not actually empty (this is harder to test, there's a helper for it, let me find it)




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       In other words - you don't need SetScalarOutput, I realize it shares more code between the two cases, but now it's at the cost of performance. FindOneMapValueIndex is the important part to refactor since that will change when this kernel is templated, but using Scalars in the Array case is strictly overhead.




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -1639,17 +1639,19 @@ in the respective option classes.
 Structural transforms
 ~~~~~~~~~~~~~~~~~~~~~
 
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| Function name       | Arity      | Input types                         | Output type      | Options class                | Notes  |
-+=====================+============+=====================================+==================+==============================+========+
-| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                              | \(1)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_flatten        | Unary      | List-like                           | List value type  |                              | \(2)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_parent_indices | Unary      | List-like                           | Int64            |                              | \(3)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions` | \(4)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| Function name       | Arity      | Input types                         | Output type      | Options class                   | Notes  |
++=====================+============+=====================================+==================+=================================+========+
+| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                                 | \(1)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_flatten        | Unary      | List-like                           | List value type  |                                 | \(2)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_parent_indices | Unary      | List-like                           | Int64            |                                 | \(3)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| map_array_lookup    | Unary      | Map                                 | Computed         | :struct:`MapArrayLookupOptions` | \(4)   |

Review comment:
       Not a problem, better to be rigorous here than have an inconsistent codebase. We just will have to be on the lookout for something that I've missed renaming.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,423 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+TEST_F(TestMapArrayLookupKernel, BooleanKey) {
+  auto true_scalar = ScalarFromJSON(boolean(), R"(true)");
+  auto map_type = map(boolean(), int32());
+  const char* input = R"(
+    [
+      [
+        [true, 99], [false, 1], [false, 2], [true, null], [false, 5],
+        [true, 8]
+      ],
+      null,
+      [
+        [false, null], [true, 67], [false, 101], [false, 1], [false, null],
+        [false, 9], [true, 80]
+      ],
+      [],
+      [
+        [false, 1], [false, 2], [false, 3], [false, 4]
+      ],
+      [
+        [true, 9], [true, 2], [true, 5], [true, 8]

Review comment:
       Ah, I didn't realize that was disallowed. Ok then, we should be good. And that means we don't have to decide what to do with null query keys. That also means we should check if the query key is null and return an error in that case.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;

Review comment:
       It's not necessarily true. `ArrayFromJSON` won't give you such lists but it's entirely valid to have such items.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());

Review comment:
       `checked_cast<const MapType&>(*batch[0].type())` is enough, a `Datum` has its own type. 




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {

Review comment:
       Is the overall boilerplate structure for a `function/kernels` correct?




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,167 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookupNested) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  auto map_array = ArrayFromJSON(type, input);
+
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto foo_all_output = ArrayFromJSON(
+      list(map(int16(), int16())),
+      "[ [[[4, 4], [5, 5]], [[8, 8], [9, 9]], [[12, 12], [13, 13]]], null, [[[4, 5], [5, "
+      "6]], [[8, 9], [9, 10]], [[12, 13], [13, 14]]], null ]");
+
+  CheckScalar("map_array_lookup", {map_array}, foo_all_output, &foo_all);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[4, 4], [5, 5]], null, [[4, 5], [5, 6]], null ]"),
+              &foo_first);
+  CheckScalar("map_array_lookup", {map_array},
+              ArrayFromJSON(map(int16(), int16()),
+                            "[ [[12, 12], [13, 13]], null, [[12, 13], [13, 14]], null ]"),
+              &foo_last);
+}
+
+template <typename Type>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, IntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {

Review comment:
       This basic structure has to be repeated for `StringArrowTypes` and `DecimalArrowTypes`, right? Also should templated tests be written for `BinaryArrowTypes` and `StringArrowTypes`?




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {

Review comment:
       This indexing check is to make sure the index is between [start, end) as `map_array.offsets()` approach is used, instead we could use something like you previously mentioned (inside `ExecMapArray`):
   ```cpp
   auto map = map_array.value_slice(map_array_idx);
   auto keys = checked_cast<const StructArray&>(*map).field(0);
   auto items = checked_cast<const StructArray&>(*map).field(1);
   ```
   Does the latter incur some performance penalty? Which approach should be used?
   
   (Ditto below)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,290 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
+                                              const Scalar& query_key_scalar,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    int64_t match_idx = -1;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              if (!from_back) {
+                match_idx = index;
+                return Status::Cancelled("Found first matching key");
+              } else {
+                match_idx = index;
+              }
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return match_idx;
+  }
+
+  static Result<std::unique_ptr<ArrayBuilder>> GetBuiltArray(
+      const Array& keys, const Array& items, const Scalar& query_key_scalar,
+      bool& found_at_least_one_key, const int64_t& start, const int64_t& end,
+      KernelContext* ctx) {
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (index < start) {
+            ++index;
+            return Status::OK();
+          } else if (index < end) {
+            if (key == query_key) {
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+            }
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        },
+        [&]() -> Status {
+          if (index < end) {
+            ++index;
+            return Status::OK();
+          } else {
+            return Status::Cancelled("End reached");
+          }
+        }));
+
+    return std::move(builder);
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+    auto offsets = std::dynamic_pointer_cast<Int32Array>(map_array.offsets());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        std::unique_ptr<ArrayBuilder> list_builder;
+        bool found_at_least_one_key = false;
+        ARROW_ASSIGN_OR_RAISE(
+            list_builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key,
+                                        start, end, ctx));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+        int64_t start = offsets->Value(map_array_idx);
+        int64_t end = offsets->Value(map_array_idx + 1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+        ARROW_ASSIGN_OR_RAISE(
+            int64_t key_match_idx,
+            FindOneMapValueIndex(*keys, *query_key, start, end, from_back));
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      ARROW_ASSIGN_OR_RAISE(
+          builder, GetBuiltArray(*keys, *items, *query_key, found_at_least_one_key, 0,
+                                 struct_array.length(), ctx));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||
+      !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_array_lookup: query_key type and MapArray key_type don't match. Expected "
+        "type: ",
+        *item_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else { /* occurrence == FIRST || LAST */
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+struct ResolveMapArrayLookup {
+  KernelContext* ctx;
+  const ExecBatch& batch;
+  Datum* out;
+
+  template <typename KeyType>
+  Status Execute() {
+    if (batch[0].kind() == Datum::SCALAR) {
+      return MapArrayLookupFunctor<KeyType>::ExecMapScalar(ctx, batch, out);
+    }
+    return MapArrayLookupFunctor<KeyType>::ExecMapArray(ctx, batch, out);
+  }
+
+  template <typename KeyType>
+  enable_if_physical_integer<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_decimal<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_base_binary<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_boolean<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+

Review comment:
       Current problems:
   - `enable_if_fixed_size_binary` conflicts with `enable_if_decimal`
   - MonthDayNanoInterval

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,294 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {

Review comment:
       Will remove these two at the end?




-- 
This is an automated message from the 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] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Finished :arrow_down:0.48% :arrow_up:0.78%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: docs/source/cpp/compute.rst
##########
@@ -1639,17 +1639,19 @@ in the respective option classes.
 Structural transforms
 ~~~~~~~~~~~~~~~~~~~~~
 
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| Function name       | Arity      | Input types                         | Output type      | Options class                | Notes  |
-+=====================+============+=====================================+==================+==============================+========+
-| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                              | \(1)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_flatten        | Unary      | List-like                           | List value type  |                              | \(2)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_parent_indices | Unary      | List-like                           | Int64            |                              | \(3)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions` | \(4)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| Function name       | Arity      | Input types                         | Output type      | Options class                   | Notes  |
++=====================+============+=====================================+==================+=================================+========+
+| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                                 | \(1)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_flatten        | Unary      | List-like                           | List value type  |                                 | \(2)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_parent_indices | Unary      | List-like                           | Int64            |                                 | \(3)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| map_array_lookup    | Unary      | Map                                 | Computed         | :struct:`MapArrayLookupOptions` | \(4)   |

Review comment:
       Not a problem, better to be rigorous here than have an inconsistent codebase. We just will have to be on the lookout for anything that I've missed renaming.




-- 
This is an automated message from the 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] lidavidm closed pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   


-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: docs/source/python/api/compute.rst
##########
@@ -504,6 +505,7 @@ Compute Options
    IndexOptions
    JoinOptions
    MakeStructOptions
+   MapArrayLookupOptions

Review comment:
       Just a reminder to revisit and implement this. 
   
   To do so, you'll need to:
   
   1. Add the options class definition to the Cython pxd: https://github.com/apache/arrow/blob/39367db2dab321dbbf4d12d2229020614b049dde/python/pyarrow/includes/libarrow.pxd#L2147-L2150
   2. Add a Python options class wrapper to the Cython code: https://github.com/apache/arrow/blob/39367db2dab321dbbf4d12d2229020614b049dde/python/pyarrow/_compute.pyx#L1317-L1333
   3. Add an import so it's in the public API: https://github.com/apache/arrow/blob/39367db2dab321dbbf4d12d2229020614b049dde/python/pyarrow/compute.py#L18
   4. Add the options class to the serialization test: https://github.com/apache/arrow/blob/39367db2dab321dbbf4d12d2229020614b049dde/python/pyarrow/tests/test_compute.py#L138 (just add it to the list there, no need to add additional assertions to the test)
   5. Add a brief test to make sure everything works (no need to add detailed tests, we just want to make sure the bindings were set up): https://github.com/apache/arrow/blob/39367db2dab321dbbf4d12d2229020614b049dde/python/pyarrow/tests/test_compute.py#L2081-L2092

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(

Review comment:
       This can't be ARROW_UNUSED anymore since AppendArraySlice above may return an actual error. Instead we should check if the status is Cancelled:
   
   ```
   auto status = Visit...;
   if (!status.ok() && !status.IsCancelled()) {
     return status;
   }
   return Status::OK();
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        std::unique_ptr<ArrayBuilder> list_builder;
+        RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                  &list_builder));
+
+        RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                      list_builder.get()));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+

Review comment:
       Here we should `builder->Reserve(batch.length)`

##########
File path: docs/source/cpp/compute.rst
##########
@@ -1639,17 +1639,19 @@ in the respective option classes.
 Structural transforms
 ~~~~~~~~~~~~~~~~~~~~~
 
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| Function name       | Arity      | Input types                         | Output type      | Options class                | Notes  |
-+=====================+============+=====================================+==================+==============================+========+
-| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                              | \(1)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_flatten        | Unary      | List-like                           | List value type  |                              | \(2)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_parent_indices | Unary      | List-like                           | Int64            |                              | \(3)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions` | \(4)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| Function name       | Arity      | Input types                         | Output type      | Options class                   | Notes  |
++=====================+============+=====================================+==================+=================================+========+
+| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                                 | \(1)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_flatten        | Unary      | List-like                           | List value type  |                                 | \(2)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_parent_indices | Unary      | List-like                           | Int64            |                                 | \(3)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions`    | \(4)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| map_array_lookup    | Unary      | MapArray                            | Computed         | :struct:`MapArrayLookupOptions` | \(5)   |

Review comment:
       Please keep these tables alphabetically sorted (sorry, you may have to renumber some items)

##########
File path: docs/source/cpp/compute.rst
##########
@@ -1639,17 +1639,19 @@ in the respective option classes.
 Structural transforms
 ~~~~~~~~~~~~~~~~~~~~~
 
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| Function name       | Arity      | Input types                         | Output type      | Options class                | Notes  |
-+=====================+============+=====================================+==================+==============================+========+
-| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                              | \(1)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_flatten        | Unary      | List-like                           | List value type  |                              | \(2)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_parent_indices | Unary      | List-like                           | Int64            |                              | \(3)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions` | \(4)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| Function name       | Arity      | Input types                         | Output type      | Options class                   | Notes  |
++=====================+============+=====================================+==================+=================================+========+
+| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                                 | \(1)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_flatten        | Unary      | List-like                           | List value type  |                                 | \(2)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_parent_indices | Unary      | List-like                           | Int64            |                                 | \(3)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions`    | \(4)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| map_array_lookup    | Unary      | MapArray                            | Computed         | :struct:`MapArrayLookupOptions` | \(5)   |

Review comment:
       ```suggestion
   | map_array_lookup    | Unary      | Map                                 | Computed         | :struct:`MapArrayLookupOptions` | \(5)   |
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        std::unique_ptr<ArrayBuilder> list_builder;
+        RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                  &list_builder));
+
+        RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                      list_builder.get()));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();

Review comment:
       This is inefficient because we're building an Array, allocating a Scalar, and then copying the data to the builder. However, a ListBuilder lets you directly append to the child array. 
   
   This should do something like
   
   ```
   auto* values_builder = checked_cast<ListBuilder*>(builder.get())->values_builder();
   builder->Append(true); // start a new list item
   FindMatchingIndices(..., [&](int64_t index) {
     values_builder->AppendArraySlice(...);
   });
   ```




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;

Review comment:
       To correctly index the `keys`/`items` `arrow::Array`'s above while accounting for which `List` they belong to below. 




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,

Review comment:
       A very preliminary attempt at resolving key and item `types`. Will such an approach work?




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]
+)";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalarNonRecursive(
+      "map_array_lookup", {map_array},
+      ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookup) {

Review comment:
       But this one fails, 
   https://github.com/apache/arrow/runs/4853174437?check_suite_focus=true#step:6:5490
   
   <details><summary> with output </summary>
   
   ```bash
   > GTEST_FILTER="TestScalarNested.MapArrayLookup*" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup*
   25: [==========] Running 2 tests from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 2 tests from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookupNonRecursive
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: [       OK ] TestScalarNested.MapArrayLookupNonRecursive (31 ms)
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:593: [
   25:   "foo",
   25:   "bar",
   25:   "hello",
   25:   "foo",
   25:   "lesgo",
   25:   "whatnow"
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:594: [
   25:   99,
   25:   1,
   25:   2,
   25:   3,
   25:   5,
   25:   8
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:595: Input: -- is_valid: all not null
   25: -- child 0 type: string
   25:   [
   25:     "foo",
   25:     "bar",
   25:     "hello",
   25:     "foo",
   25:     "lesgo",
   25:     "whatnow"
   25:   ]
   25: -- child 1 type: int32
   25:   [
   25:     99,
   25:     1,
   25:     2,
   25:     3,
   25:     5,
   25:     8
   25:   ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:649: MapArrayLookup Scalar result: [
   25:   99,
   25:   3
   25: ] Type: list<item: int32>
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: unknown file: Failure
   25: C++ exception with description "std::bad_cast" thrown in the test body.
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (3 ms)
   25: [----------] 2 tests from TestScalarNested (35 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 2 tests from 1 test suite ran. (35 ms total)
   25: [  PASSED  ] 1 test.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.13 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.13 sec*proc (1 test)
   unittest         =   0.13 sec*proc (1 test)
   
   Total Test time (real) =   0.13 sec
   
   The following tests FAILED:
   	 25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   
   Am getting this same error as above for every option (`FIRST`, `LAST`, `ALL`):
   ```bash
   unknown file: Failure
   C++ exception with description "std::bad_cast" thrown in the test body.
   ```
   and am guessing I'm doing something wrong in `ResolveMapArrayLookupType`?




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());

Review comment:
       As per your comment below, `map_scalar.is_valid` needs to be checked _before_ casting, but in that case, how could a `NullScalar` of an appropriate type be made (which requires `item_type`)?




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {

Review comment:
       Yes of course, just wanted to check that at least the basic functionality worked. 




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;

Review comment:
       Ignore `All` for now. Let's get `First` to work and then we can see how `All` works.
   
   The logic here looks reasonable, though maybe I'm missing something as you should have one output per input row and the test cases below don't reflect that.




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);

Review comment:
       The length of the current `List`, aka, the number of keys to check in the `keys` `arrow::Array` above (which is _continous_ in memory and are separated by offsets).
   Have I understood this correctly, and is using the `map_array.value_length(map_array_idx)` the correct way to obtain it?




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

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

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



[GitHub] [arrow] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -17,6 +17,7 @@
 
 // Vector kernels involving nested types
 
+#include <iostream>

Review comment:
       Note you can use `arrow/util/logging.h` and `ARROW_LOG(WARNING) << ...;` (that way you don't have to deal with std::endl/the header might already be included, also, you'll get line numbers)




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},

Review comment:
       Using `CheckScalarNonRecursive` instead of `CheckScalar`, as using the latter caused these errors:
   
   <details><summary> Using CheckScalar </summary>
   
   ```bash
   ❯ GTEST_FILTER="TestScalarNested.MapArrayLookup" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup
   25: [==========] Running 1 test from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 1 test from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: Key being checked: foo
   25: Item at key: 101
   25: Value being appended: 101
   25: Key being checked: foo
   25: Item at key: 22
   25: Value being appended: 22
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -1, +1 @@
   25: +3
   25: Expected:
   25:   [
   25:     99
   25:   ]
   25: Actual:
   25:   [
   25:     99,
   25:     3
   25:   ]
   25: map type found!
   25: Value type: int32
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -3
   25: Expected:
   25:   [
   25:     3
   25:   ]
   25: Actual:
   25:   []
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -101
   25: -22
   25: +99
   25: +3
   25: Expected:
   25:   [
   25:     101,
   25:     22
   25:   ]
   25: Actual:
   25:   [
   25:     99,
   25:     3
   25:   ]
   25: map type found!
   25: Value type: int32
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:223: Failure
   25: Failed
   25: Got:
   25:   [
   25:     [
   25:       99,
   25:       3
   25:     ],
   25:     [
   25:       99,
   25:       3
   25:     ]
   25:   ]
   25: Expected:
   25:   [
   25:     [
   25:       99
   25:     ],
   25:     [
   25:       3,
   25:       101,
   25:       22
   25:     ]
   25:   ]
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:119: Failure
   25: Expected equality of these values:
   25:   input.array()->length
   25:     Which is: 4
   25:   expected->length()
   25:     Which is: 1
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (9 ms)
   25: [----------] 1 test from TestScalarNested (9 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 1 test from 1 test suite ran. (9 ms total)
   25: [  PASSED  ] 0 tests.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.08 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.08 sec*proc (1 test)
   unittest         =   0.08 sec*proc (1 test)
   
   Total Test time (real) =   0.08 sec
   
   The following tests FAILED:
            25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   
   Also, what would a kernel that takes `Type::MAP` of shape `ValueDescr::SCALAR`, that is, a Scalar kernel, look like 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       For LAST, you could keep track of the index outside the "loop" and simply overwrite it each time (though that is perhaps not as efficient), returning the final index. We could later define a visitor that goes in reverse. 

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       I would say, don't use SetScalarOutput and stick to AppendArraySlice. Allocating a Scalar for each item will be expensive especially for things like strings where you have to copy the data to a Scalar only to copy it again into an 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       But offsets are only needed in the Array case, as MapScalar -> StructArray -> field(0, 1) approach for the Scalar case give the _appropriate_(corresponding just to the given map element from the MapArray) keys/items anyways.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));

Review comment:
       ```suggestion
           auto items = map_array.value_slice(map_array_idx);
           auto keys = checked_cast<const StructArray*>(*items).field(0);
           ARROW_ASSIGN_OR_RAISE(auto index, FindOneMapValueIndex(*keys, ...));
           // Append the value (adjusting index since index is relative to the current item)
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,294 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+template <typename KeyType>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, PhysicalIntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {
+  std::shared_ptr<DataType> type = default_type_instance<TypeParam>();
+
+  auto map_type = map(type, utf8());
+  const char* input = R"(
+    [
+      [ 
+        [0, "zero"], [1, "first_one"], [2, "two"], [1, null], [3, "three"], [1, "second_one"],
+        [1, "last_one"]
+      ],
+      null,
+      [ 
+        [0, "zero_hero"], [9, "almost_six"], [1, "the_dumb_one"], [7, "eleven"],
+        [1, "the_chosen_one"], [42, "meaning of life?"], [1, "just_one"],
+        [1, "no more ones!"]
+      ],
+      [
+        [4, "this"], [6, "has"], [8, "no"], [2, "ones"]
+      ],
+      [
+        [1, "this"], [1, "should"], [1, "also"], [1, "be"], [1, "null"]
+      ],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(map_type, input);
+  auto map_array_tweaked = TweakValidityBit(map_array, 4, false);
+
+  auto expected_all = ArrayFromJSON(list(utf8()), R"(
+                          [
+                            ["first_one", null, "second_one", "last_one"],
+                            null,
+                            ["the_dumb_one", "the_chosen_one", "just_one", "no more ones!"],
+                            null,
+                            null,
+                            null
+                          ])");
+  auto expected_first =
+      ArrayFromJSON(utf8(), R"(["first_one", null, "the_dumb_one", null, null, null])");
+  auto expected_last =
+      ArrayFromJSON(utf8(), R"(["last_one", null, "no more ones!", null, null, null])");
+
+  CheckMapArrayLookupWithDifferentOptions(map_array_tweaked,
+                                          MakeScalar(type, 1).ValueOrDie(), expected_all,
+                                          expected_first, expected_last);
+}
+template <typename KeyType>
+class TestMapArrayLookupDecimalKeys : public ::testing ::Test {
+ protected:
+  std::shared_ptr<DataType> GetType() {

Review comment:
       Note that most tests call this `std::shared_ptr<DataType> type_singleton() const`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));
+      }
+    }
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    ARROW_ASSIGN_OR_RAISE(out->value, GetScalarOutput(ctx, map_scalar));
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||
+      !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_array_lookup: query_key type and MapArray key_type don't match. Expected "
+        "type: ",
+        *item_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else /* occurrence == FIRST || LAST */ {
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+void AddMapArrayLookupKernels(ScalarFunction* func) {
+  for (const auto shape : {ValueDescr::ARRAY, ValueDescr::SCALAR}) {
+    ScalarKernel kernel({InputType(Type::MAP, shape)},
+                        OutputType(ResolveMapArrayLookupType),
+                        shape == ValueDescr::ARRAY ? MapArrayLookupFunctor::ExecMapArray
+                                                   : MapArrayLookupFunctor::ExecMapScalar,

Review comment:
       In order to use `VisitArrayValuesInline` this will have to change since you need to template `MapArrayLookupFunctor` with a `template <typename Type>`. Normally we would register one kernel for each supported type here, however, that breaks down because the input type needs to be parametric and we don't support that.
   
   We could make a custom type mapper, however, instead, I think we can do the following and use a visitor
   
   ```
   ScalarKernel kernel(InputType(Type::MAP), ..., ResolveMapArrayLookup::Exec);
   
   struct ResolveMapArrayLookup {
     KernelContext* ctx;
     const ExecBatch& batch;
     Datum* out;
   
     template <typename Type>
     Status Execute() {
       if (batch[0].kind() == Datum::SCALAR) {
         return MapArrayLookupFunctor::ExecMapScalar(ctx, batch, out);
       }
       return MapArrayLookupFunctor::ExecMapArray(ctx, batch, out)
     }
   
     template <typename Type>
     enable_if_physical_integer<Type, Status> Visit(const Type& type) {
       return Execute<Type>();
     }
   
     // ...
   
     Status Visit(const DataType& type) { return Status::TypeError(...); }
   
     static Status Exec(...) {
       ResolveMapArrayLookup visitor{ctx, batch, out};
       return VisitTypeInline(checked_cast<const MapType&>(*batch[0].type()), *this);
     }
   };
   ```
   Another visitor example: https://github.com/apache/arrow/blob/91e3ac53e2e21736ce6291d73fc37da6fa21259d/cpp/src/arrow/compute/kernels/scalar_set_lookup.cc#L332
   
   The custom type mapper would look something like this, I think:
   https://github.com/apache/arrow/blob/8d0efd98bd4c4f2d181e01c02a66f924cb3e4c94/cpp/src/arrow/compute/kernel.cc#L176
   
   You would define a custom TypeMatcher that checks the map key type, then register it as follows
   
   ```
   for (const auto& type : IntTypes()) {
     ScalarKernel kernel({InputType(MapKeyTypeMatcher(type)), ..., GeneratePhysicalNumeric<MapArrayLookupFunctor>(type));
   }
   ```
   
   (I think I gave examples of this earlier)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       Here perhaps: https://github.com/apache/arrow/blob/91e3ac53e2e21736ce6291d73fc37da6fa21259d/cpp/src/arrow/compute/kernels/scalar_if_else.cc#L2565

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,294 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {
+  auto type = map(utf8(), map(int16(), int16()));
+  const char* input = R"(
+    [
+      [
+        [
+          "just",
+          [[0, 0], [1, 1]]
+        ],
+        [
+          "random",
+          [[2, 2], [3, 3]]
+        ],
+        [
+          "foo",
+          [[4, 4], [5, 5]]
+        ],
+        [
+          "values",
+          [[6, 6], [7, 7]]
+        ],
+        [
+          "foo",
+          [[8, 8], [9, 9]]
+        ],
+        [
+          "point",
+          [[10, 10], [11, 11]]
+        ],
+        [
+          "foo",
+          [[12, 12], [13, 13]]
+        ]
+      ],
+      null,
+      [
+        [
+          "yet",
+          [[0, 1], [1, 2]]
+        ],
+        [
+          "more",
+          [[2, 3], [3, 4]]
+        ],
+        [
+          "foo",
+          [[4, 5], [5, 6]]
+        ],
+        [
+          "random",
+          [[6, 7], [7, 8]]
+        ],
+        [
+          "foo",
+          [[8, 9], [9, 10]]
+        ],
+        [
+          "values",
+          [[10, 11], [11, 12]]
+        ],
+        [
+          "foo",
+          [[12, 13], [13, 14]]
+        ]
+      ],
+      []
+    ]
+  )";
+  const auto map_array = ArrayFromJSON(type, input);
+  const auto expected_all = ArrayFromJSON(list(map(int16(), int16())), R"(
+                                [
+                                  [
+                                    [[4, 4], [5, 5]], [[8, 8], [9, 9]],
+                                    [[12, 12], [13, 13]]
+                                  ],
+                                  null,
+                                  [
+                                    [[4, 5], [5, 6]], [[8, 9], [9, 10]],
+                                    [[12, 13], [13, 14]]
+                                  ],
+                                  null
+                                ])");
+  const auto expected_first = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[4, 4], [5, 5]],
+                                  null,
+                                  [[4, 5], [5, 6]],
+                                  null
+                                ])");
+  const auto expected_last = ArrayFromJSON(map(int16(), int16()), R"(
+                                [
+                                  [[12, 12], [13, 13]],
+                                  null,
+                                  [[12, 13], [13, 14]],
+                                  null
+                                ])");
+  CheckMapArrayLookupWithDifferentOptions(map_array, MakeScalar("foo"), expected_all,
+                                          expected_first, expected_last);
+}
+
+template <typename KeyType>
+class TestMapArrayLookupIntegralKeys : public ::testing ::Test {};
+
+TYPED_TEST_SUITE(TestMapArrayLookupIntegralKeys, PhysicalIntegralArrowTypes);
+
+TYPED_TEST(TestMapArrayLookupIntegralKeys, StringItems) {
+  std::shared_ptr<DataType> type = default_type_instance<TypeParam>();
+
+  auto map_type = map(type, utf8());

Review comment:
       This could be refactored into a `type_singleton()` method for consistency with below.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));

Review comment:
       This still does unnecessary boxing/unboxing, you could improve it further if `FindOneMapValueIndex` took `const Array& keys, int64_t start`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,294 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+void CheckMapArrayLookupWithDifferentOptions(
+    const std::shared_ptr<Array>& map, const std::shared_ptr<Scalar>& query_key,
+    const std::shared_ptr<Array>& expected_all,
+    const std::shared_ptr<Array>& expected_first,
+    const std::shared_ptr<Array>& expected_last) {
+  MapArrayLookupOptions all_matches(query_key, MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions first_matches(query_key, MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions last_matches(query_key, MapArrayLookupOptions::LAST);
+
+  CheckScalar("map_array_lookup", {map}, expected_all, &all_matches);
+  CheckScalar("map_array_lookup", {map}, expected_first, &first_matches);
+  CheckScalar("map_array_lookup", {map}, expected_last, &last_matches);
+}
+
+class TestMapArrayLookupKernel : public ::testing::Test {};
+
+TEST_F(TestMapArrayLookupKernel, Basic) {
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+    [
+      [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lets go", 5], ["what now?", 8]],
+      null,
+      [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null],
+      ["foo", 22]],
+      []
+    ])";
+  auto map_array = ArrayFromJSON(type, input);
+  CheckMapArrayLookupWithDifferentOptions(
+      map_array, MakeScalar("foo"),
+      ArrayFromJSON(list(int32()), R"([[99, 3], null, [101, 22], null])"),
+      ArrayFromJSON(int32(), R"([99, null, 101, null])"),
+      ArrayFromJSON(int32(), R"([3, null, 22, null])"));
+}
+
+TEST_F(TestMapArrayLookupKernel, NestedItems) {

Review comment:
       We probably don't need `Basic` or `NestedItems` anymore but we could keep `TestMapArrayLookupKernel` for testing boolean, fixed-size string, and month-day-nano-interval key types. (Also, you can just use `TEST` instead of `TEST_F` and remove the `TestMapArrayLookupKernel` class here.)

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const std::shared_ptr<Scalar>& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    std::shared_ptr<Scalar> output;
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        output = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(output, MakeScalar(list(items->type()), result));
+      }
+    }
+
+    else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(*keys, *query_key, 0, struct_array.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(output, items->GetScalar(key_match_idx));
+      } else {
+        output = MakeNullScalar(items->type());
+      }
+    }
+    return output;
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> scalar,
+                              map_array.GetScalar(map_array_idx));
+        auto map_scalar = std::static_pointer_cast<MapScalar>(scalar);
+        ARROW_ASSIGN_OR_RAISE(auto scalar_output, GetScalarOutput(ctx, *map_scalar));
+        RETURN_NOT_OK(builder->AppendScalar(*scalar_output));

Review comment:
       And then you can inline `ExecMapScalar` back into the scalar case. The point of this is to avoid boxing/unboxing individual values into scalars.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       - So, this was the basic structure I was thinking of,
   ```cpp
     template <typename KeyType>
     struct MapArrayLookupFunctor {
     .
     .
     .
     const KeyType query_key = UnboxScalar<KeyType>::Unbox(*options.query_key);
     ...
     const std::shared_ptr<Array> keys = struct_array.field(0);
     ...
     
     VisitArrayValuesInline<KeyType>(
       *keys->data(),
       [&](KeyType key) -> Status {
         if (key == query_key) {
           if (occurrence == FIRST) {
             ...
             return Status::Cancelled("Found key");
           } else /*occurrence == ALL*/ { 
               ...
               return Status::OK(); 
           }
         } else { return Status::OK(); }
        },
       [&]() -> Status {
         ...
         return Status::OK();
       }
     );
     ``` 
   - But as the values are visited _inline_, how would we deal with the `LAST` option (iterating from the back)?
   - And just so I'm clear, when you say templating the kernel, you mean templating for the `KeyType`, right?
   




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

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

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



[GitHub] [arrow] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Failed] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,264 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, list_builder->Finish());
+      out->value = result->data();
+    } else { /* occurrence == FIRST || LAST */
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+      RETURN_NOT_OK(builder->Reserve(batch.length));
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      RETURN_NOT_OK(FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+        found_at_least_one_key = true;
+        RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), index, 1));
+        return Status::OK();
+      }));
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapLookupType(KernelContext* ctx,
+                                        const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key) {
+    return Status::Invalid("map_lookup: query_key can't be empty.");
+  } else if (!options.query_key->is_valid) {
+    return Status::Invalid("map_lookup: query_key can't be null.");
+  } else if (!options.query_key->type || !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_lookup: query_key type and Map key_type don't match. Expected "
+        "type: ",
+        *key_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else { /* occurrence == FIRST || LAST */
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+struct ResolveMapLookup {
+  KernelContext* ctx;
+  const ExecBatch& batch;
+  Datum* out;
+
+  template <typename KeyType>
+  Status Execute() {
+    if (batch[0].kind() == Datum::SCALAR) {
+      return MapLookupFunctor<KeyType>::ExecMapScalar(ctx, batch, out);
+    }
+    return MapLookupFunctor<KeyType>::ExecMapArray(ctx, batch, out);
+  }
+
+  template <typename KeyType>
+  enable_if_physical_integer<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_decimal<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_base_binary<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_boolean<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_same<KeyType, FixedSizeBinaryType, Status> Visit(const KeyType& key) {
+    return Execute<KeyType>();
+  }

Review comment:
       Does this work if it's just a normal overload? `Status Visit(const FixedSizeBinaryType& key)`

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,264 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, list_builder->Finish());
+      out->value = result->data();
+    } else { /* occurrence == FIRST || LAST */
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+      RETURN_NOT_OK(builder->Reserve(batch.length));
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      RETURN_NOT_OK(FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+        found_at_least_one_key = true;
+        RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), index, 1));
+        return Status::OK();
+      }));
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapLookupType(KernelContext* ctx,
+                                        const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key) {
+    return Status::Invalid("map_lookup: query_key can't be empty.");
+  } else if (!options.query_key->is_valid) {
+    return Status::Invalid("map_lookup: query_key can't be null.");
+  } else if (!options.query_key->type || !options.query_key->type->Equals(key_type)) {

Review comment:
       Hmm, `query_key->type` should never be nullptr, so the check is redundant. (And if the check weren't redundant, we'd have undefined behavior below on line 611 when we dereference the pointer.)




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,276 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));

Review comment:
       This needs both the `list_builder` and the `value_builder`, so haven't added this to the helper `BuildItemsArray` above as it'll take away from its simplicity. Thus, the `BuildItemsArray` could be inlined if need be as it's only being used in the Scalar case now.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        std::unique_ptr<ArrayBuilder> list_builder;
+        RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                  &list_builder));
+
+        RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                      list_builder.get()));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+      RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                    builder.get()));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||

Review comment:
       No, this tests if the shared_ptr is null, but a valid Scalar may contain a null value (Scalar.is_valid) - that needs to be checked as well

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        std::unique_ptr<ArrayBuilder> list_builder;
+        RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                  &list_builder));
+
+        RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                      list_builder.get()));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+      RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                    builder.get()));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||

Review comment:
       (Also, we should break out the error separately for `!query_key || !query_key->is_valid` so that we give a relevant error message. This will crash if we give a nullptr query_key since the error message tries to read query_key->type.)




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,271 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        std::unique_ptr<ArrayBuilder> list_builder;
+        RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                  &list_builder));
+
+        RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                      list_builder.get()));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+      RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                    builder.get()));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key || !options.query_key->type ||

Review comment:
       This does the query_key `null` checking part, right? @lidavidm 
   The Errors test also tests 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       And yes, templating is for the type of the key.




-- 
This is an automated message from the 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] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Failed] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Failed] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] ursabot edited a comment on pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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


   Benchmark runs are scheduled for baseline = 5ab41120378d1944cfb607ad745cd3c6e6e4f71a and contender = 76decf67328d40a314a24adadc2b8f3e2190340f. 76decf67328d40a314a24adadc2b8f3e2190340f is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Finished :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/c10b3e36e6dc44aaaf81af34c58a5152...5f10dce517b94270ab59373a668ada18/)
   [Scheduled] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/c79411b590b84744bfad1f17c5ac2969...37f2d1f867e94a9297fcfd6c57e00f20/)
   [Scheduled] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/47bddeafc87049328d264dc17faaaf45...6a0b46f2a5844e789fd2a1d22598e6bb/)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python. Runs only benchmarks with cloud = True
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;

Review comment:
       Currently just outputting an `Array` of type same as the `item_type` in the `MapArray`. With just one element for the `Occurence::First` option.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);
+      for (int32_t key_idx_to_check = last_key_idx_checked;
+           key_idx_to_check < last_key_idx_checked + list_struct_len;
+           ++key_idx_to_check) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                              keys->GetScalar(key_idx_to_check));
+        if (key->Equals(*query_key)) {
+          std::cout << "Key being checked: " << key->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> item,
+                                items->GetScalar(key_idx_to_check));
+          std::cout << "Value at key: " << item->ToString() << "\n";
+          ARROW_ASSIGN_OR_RAISE(auto value,
+                                item->CastTo(map_array.map_type()->item_type()));
+
+          std::cout << "Item being appended: " << value->ToString() << "\n";
+          RETURN_NOT_OK(builder->AppendScalar(*value));
+
+          if (occurence == MapArrayLookupOptions::First) {
+            found_one_key = true;
+            break;
+          }
+        }
+      }
+      if (found_one_key && occurence == MapArrayLookupOptions::First) break;
+
+      // new index from where to start checking
+      last_key_idx_checked += list_struct_len;
+    }
+    // For now, handling 'Last' and 'All' occurence options as same
+    // TODO: Handle 'Last' option.
+    ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+    out->value = result->data();
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,

Review comment:
       A very preliminary attempt at resolving key and item `types`. Will this always correctly work?




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurence;
+
+    std::unique_ptr<ArrayBuilder> builder;
+    RETURN_NOT_OK(
+        MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+    int32_t last_key_idx_checked = 0;
+
+    // aka, number of {key, value} pairs in the current map
+    int32_t list_struct_len;
+    bool found_one_key = false;
+    for (int32_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      // Number of Struct('s) = {key, value} in the list at the current index
+      list_struct_len = map_array.value_length(map_array_idx);

Review comment:
       The length of the current `List`, aka, the number of `keys` to check in the `keys` `arrow::Array` above (which is _continous_ in memory and are separated by offsets).
   Have I understood this correctly, and is using the `map_array.value_length(map_array_idx)` the correct way to obtain it?




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

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

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



[GitHub] [arrow] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {

Review comment:
       This test passes.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]

Review comment:
       (believing too much in my automatic linter)




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());

Review comment:
       I would suggest looking at this in a debugger if you haven't already, just to see what this case looks like.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]
+)";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalarNonRecursive(
+      "map_array_lookup", {map_array},
+      ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookup) {

Review comment:
       But this one fails, 
   https://github.com/apache/arrow/runs/4853174437?check_suite_focus=true#step:6:5490
   
   <details><summary> with output </summary>
   
   ```bash
   > GTEST_FILTER="TestScalarNested.MapArrayLookup*" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup*
   25: [==========] Running 2 tests from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 2 tests from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookupNonRecursive
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:675: array[int32]
   25: [       OK ] TestScalarNested.MapArrayLookupNonRecursive (31 ms)
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: array[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:593: [
   25:   "foo",
   25:   "bar",
   25:   "hello",
   25:   "foo",
   25:   "lesgo",
   25:   "whatnow"
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:594: [
   25:   99,
   25:   1,
   25:   2,
   25:   3,
   25:   5,
   25:   8
   25: ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:595: Input: -- is_valid: all not null
   25: -- child 0 type: string
   25:   [
   25:     "foo",
   25:     "bar",
   25:     "hello",
   25:     "foo",
   25:     "lesgo",
   25:     "whatnow"
   25:   ]
   25: -- child 1 type: int32
   25:   [
   25:     99,
   25:     1,
   25:     2,
   25:     3,
   25:     5,
   25:     8
   25:   ]
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:649: MapArrayLookup Scalar result: [
   25:   99,
   25:   3
   25: ] Type: list<item: int32>
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/scalar_nested.cc:678: scalar[list<item: int32>]
   25: unknown file: Failure
   25: C++ exception with description "std::bad_cast" thrown in the test body.
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (3 ms)
   25: [----------] 2 tests from TestScalarNested (35 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 2 tests from 1 test suite ran. (35 ms total)
   25: [  PASSED  ] 1 test.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.13 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.13 sec*proc (1 test)
   unittest         =   0.13 sec*proc (1 test)
   
   Total Test time (real) =   0.13 sec
   
   The following tests FAILED:
   	 25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   
   Am getting this same error as above for every option (`FIRST`, `LAST`, `ALL`):
   ```bash
   unknown file: Failure
   C++ exception with description "std::bad_cast" thrown in the test body.
   ```
   and am guessing I'm doing something wrong in `ResolveMapArrayLookupType`




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([

Review comment:
       Nothing to be sorry for, just wanted to point this out.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+      return Status::OK();
+    }
+
+    const std::shared_ptr<Scalar> query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    // for debugging
+    ARROW_LOG(WARNING) << keys->ToString();
+    ARROW_LOG(WARNING) << items->ToString();
+    ARROW_LOG(WARNING) << "Input: " << struct_array.ToString();
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::FIRST) {
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          ARROW_ASSIGN_OR_RAISE(auto result, items->GetScalar(idx));
+          out->value = result;

Review comment:
       ```suggestion
             ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(idx));
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;

Review comment:
       Instead of declaring these up here, why not just declare them inside the loop?

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+      return Status::OK();
+    }
+
+    const std::shared_ptr<Scalar> query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    // for debugging
+    ARROW_LOG(WARNING) << keys->ToString();
+    ARROW_LOG(WARNING) << items->ToString();
+    ARROW_LOG(WARNING) << "Input: " << struct_array.ToString();
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::FIRST) {
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          ARROW_ASSIGN_OR_RAISE(auto result, items->GetScalar(idx));
+          out->value = result;
+          return Status::OK();
+        }
+      }
+      out->value = MakeNullScalar(items->type());
+    }
+
+    else if (occurrence == MapArrayLookupOptions::Occurrence::LAST) {
+      int32_t last_key_idx_match = -1;
+
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {

Review comment:
       Why not just iterate backwards?

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }

Review comment:
       The inner loop logic is very similar to the scalar logic; you could probably pull out a helper function that works in terms of indices and array offsets, and that way you could share most of the logic between the two cases.
   
   `Result<int64_t> FindOneMapValue(const Array& keys, const Array& values, const Scalar& key, int64_t start_offset, int64_t slot_length)` or so

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {

Review comment:
       Note that CheckScalar calls CheckScalarNonRecursive for you so this test isn't necessary once the one below is fixed.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]

Review comment:
       nit, but try to keep the indent lined up here?

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+      return Status::OK();
+    }
+
+    const std::shared_ptr<Scalar> query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    // for debugging
+    ARROW_LOG(WARNING) << keys->ToString();
+    ARROW_LOG(WARNING) << items->ToString();
+    ARROW_LOG(WARNING) << "Input: " << struct_array.ToString();
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::FIRST) {
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          ARROW_ASSIGN_OR_RAISE(auto result, items->GetScalar(idx));
+          out->value = result;
+          return Status::OK();
+        }
+      }
+      out->value = MakeNullScalar(items->type());
+    }
+
+    else if (occurrence == MapArrayLookupOptions::Occurrence::LAST) {
+      int32_t last_key_idx_match = -1;
+
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          last_key_idx_match = idx;
+        }
+      }
+      if (last_key_idx_match == -1) {
+        out->value = MakeNullScalar(items->type());
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, items->GetScalar(last_key_idx_match));
+
+        ARROW_LOG(WARNING) << result->ToString() << " Type: " << *result->type;
+
+        out->value = result;
+      }
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::ALL) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+
+      bool found_one_key = false;
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          found_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), idx, 1));
+        }
+      }
+      if (!found_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+
+        ARROW_ASSIGN_OR_RAISE(auto scalar_list, MakeScalar(list(items->type()), result));
+        ARROW_LOG(WARNING) << "MapArrayLookup Scalar result: " << result->ToString()
+                           << " Type: " << *scalar_list->type;
+        out->value = scalar_list;
+      }
+    }
+
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key->type->Equals(key_type)) {

Review comment:
       nit, but I would also check for `!options.query_key || !options.query_key->type` in case someone didn't initialize things properly

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]
+)";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalarNonRecursive(
+      "map_array_lookup", {map_array},
+      ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookup) {

Review comment:
       Have you tried using a debugger? It can show you which cast failed.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,56 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookupNonRecursive) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::ALL);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::FIRST);
+  MapArrayLookupOptions foo_last(MakeScalar("foo"), MapArrayLookupOptions::LAST);
+
+  auto type = map(utf8(), int32());
+  const char* input = R"(
+[
+    [["foo", 99], ["bar", 1], ["hello", 2], ["foo", 3], ["lesgo", 5], ["whatnow", 8]],
+    null,
+    [["nothing", null], ["hat", null], ["foo", 101], ["sorry", 1], ["dip", null], ["foo", 22]],
+    []
+  ]
+)";
+  auto map_array = ArrayFromJSON(type, input);
+
+  CheckScalarNonRecursive(
+      "map_array_lookup", {map_array},
+      ArrayFromJSON(list(int32()), "[[99, 3], null, [101, 22], null]"), &foo_all);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[99, null, 101, null]"), &foo_first);
+  CheckScalarNonRecursive("map_array_lookup", {map_array},
+                          ArrayFromJSON(int32(), "[3, null, 22, null]"), &foo_last);
+}
+
+TEST(TestScalarNested, MapArrayLookup) {

Review comment:
       In this case it seems you need to check `map_scalar.is_valid` _before_ casting anything since in that case `map_scalar.value == nullptr`.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;

Review comment:
       You could check `map_array.IsValid` here and skip the inner loop entirely if the slot is null.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      int32_t last_key_idx_match;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        last_key_idx_match = -1;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            last_key_idx_match = key_idx_to_check;
+          }
+        }
+        if (last_key_idx_match == -1) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), last_key_idx_match, 1));
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else /* occurrence == MapArrayLookupOptions::Occurrence::All) */ {
+      std::unique_ptr<ArrayBuilder> builder;
+      std::unique_ptr<ArrayBuilder> list_builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        if (list_struct_len > 0) {
+          RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(),
+                                    &list_builder));
+        }
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(
+                list_builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        } else {
+          ARROW_ASSIGN_OR_RAISE(auto list_result, list_builder->Finish());
+          RETURN_NOT_OK(builder->AppendScalar(ListScalar(list_result)));
+        }
+        list_builder->Reset();
+
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+      return Status::OK();
+    }
+
+    const std::shared_ptr<Scalar> query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    // for debugging
+    ARROW_LOG(WARNING) << keys->ToString();
+    ARROW_LOG(WARNING) << items->ToString();
+    ARROW_LOG(WARNING) << "Input: " << struct_array.ToString();
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::FIRST) {
+      for (int32_t idx = 0; idx < struct_array.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys->GetScalar(idx));
+
+        if (key->Equals(*query_key)) {
+          ARROW_ASSIGN_OR_RAISE(auto result, items->GetScalar(idx));
+          out->value = result;

Review comment:
       And ditto below.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,274 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::shared_ptr<arrow::Array> keys = map_array.keys();
+    std::shared_ptr<arrow::Array> items = map_array.items();
+
+    const auto& query_key = options.query_key;
+    const auto& occurence = options.occurrence;
+
+    if (occurence == MapArrayLookupOptions::Occurrence::FIRST) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+
+      int32_t last_key_idx_checked = 0;
+
+      // aka, number of {key, value} pairs in the current map
+      int32_t list_struct_len;
+      bool found_one_key;
+
+      for (int32_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        // Number of Struct('s) = {key, value} in the list at the current index
+        list_struct_len = map_array.value_length(map_array_idx);
+        found_one_key = false;
+
+        for (int32_t key_idx_to_check = last_key_idx_checked;
+             key_idx_to_check < last_key_idx_checked + list_struct_len;
+             ++key_idx_to_check) {
+          ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key,
+                                keys->GetScalar(key_idx_to_check));
+
+          if (key->Equals(*query_key)) {
+            found_one_key = true;
+            RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_idx_to_check, 1));
+            break;
+          }
+        }
+        if (!found_one_key) {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+        // new index from where to start checking
+        last_key_idx_checked += list_struct_len;
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+
+    } else if (occurence == MapArrayLookupOptions::Occurrence::LAST) {

Review comment:
       It seems you could combine FIRST and LAST, you would just have to branch to determine the direction of iteration in the inner loop.




-- 
This is an automated message from the 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 #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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






-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +430,97 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    MapArray map_array(batch[0].array());
+
+    // Offset differences will tell the number of Strcut = {key, value} pairs
+    // present in the current list.
+    // const std::shared_ptr<arrow::Buffer> offsets = map_array.value_offsets();

Review comment:
       Initially thought of using `offsets` to know which `<key: item>` pairs belong to which `List`.




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested_test.cc
##########
@@ -225,6 +225,30 @@ TEST(TestScalarNested, StructField) {
   }
 }
 
+TEST(TestScalarNested, MapArrayLookup) {
+  MapArrayLookupOptions foo_all(MakeScalar("foo"), MapArrayLookupOptions::All);
+  MapArrayLookupOptions foo_first(MakeScalar("foo"), MapArrayLookupOptions::First);
+  auto type = map(utf8(), int32());
+
+  auto keys = ArrayFromJSON(utf8(), R"([
+    "foo", "bar", "hello", "foo", "lesgo", "whatnow",
+    "nothing", "hat", "foo", "sorry", "dip", "foo"
+  ])");
+  auto items = ArrayFromJSON(int16(), R"([
+    99,    1,    2,  3,  5,    8,
+    null, null, 101,  1,  null, 22
+  ])");
+  auto offsets = ArrayFromJSON(int32(), "[0, 6, 6, 12, 12]")->data()->buffers[1];
+  auto null_bitmap = ArrayFromJSON(boolean(), "[1, 0, 1, 1]")->data()->buffers[1];
+
+  MapArray map_array(type, 4, offsets, keys, items, null_bitmap, 1, 0);
+
+  CheckScalarNonRecursive("map_array_lookup", {map_array},

Review comment:
       Using `CheckScalarNonRecursive` instead of `CheckScalar`, as using the latter caused these errors:
   
   <details><summary> Using CheckScalar </summary>
   
   ```bash
   ❯ GTEST_FILTER="TestScalarNested.MapArrayLookup" ctest -R "arrow-compute-scalar-test" -V
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   UpdateCTestConfiguration  from :/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/DartConfiguration.tcl
   Test project /home/dhruv/repos/arrow/cpp/out/build/ninja-debug
   Constructing a list of tests
   Done constructing a list of tests
   Updating test list for fixtures
   Added 0 tests to meet fixture requirements
   Checking test dependency graph...
   Checking test dependency graph end
   test 25
       Start 25: arrow-compute-scalar-test
   
   25: Test command: /home/dhruv/repos/arrow/cpp/build-support/run-test.sh "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug" "test" "/home/dhruv/repos/arrow/cpp/out/build/ninja-debug/debug//arrow-compute-scalar-test"
   25: Test timeout computed to be: 10000000
   25: Running arrow-compute-scalar-test, redirecting output into /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/build/test-logs/arrow-compute-scalar-test.txt (attempt 1/1)
   25: Running main() from ../googletest/src/gtest_main.cc
   25: Note: Google Test filter = TestScalarNested.MapArrayLookup
   25: [==========] Running 1 test from 1 test suite.
   25: [----------] Global test environment set-up.
   25: [----------] 1 test from TestScalarNested
   25: [ RUN      ] TestScalarNested.MapArrayLookup
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: Key being checked: foo
   25: Item at key: 101
   25: Value being appended: 101
   25: Key being checked: foo
   25: Item at key: 22
   25: Value being appended: 22
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:86: Failure
   25: Failed
   25: '_error_or_value12.status()' failed with NotImplemented: Function 'map_array_lookup' has no kernel matching input types (scalar[map<string, int32>])
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/function.cc:224  DispatchBest(&inputs)
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -1, +1 @@
   25: +3
   25: Expected:
   25:   [
   25:     99
   25:   ]
   25: Actual:
   25:   [
   25:     99,
   25:     3
   25:   ]
   25: map type found!
   25: Value type: int32
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -3
   25: Expected:
   25:   [
   25:     3
   25:   ]
   25: Actual:
   25:   []
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:134: Failure
   25: Failed
   25:
   25: @@ -0, +0 @@
   25: -101
   25: -22
   25: +99
   25: +3
   25: Expected:
   25:   [
   25:     101,
   25:     22
   25:   ]
   25: Actual:
   25:   [
   25:     99,
   25:     3
   25:   ]
   25: map type found!
   25: Value type: int32
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: Key being checked: foo
   25: Item at key: 3
   25: Value being appended: 3
   25: /home/dhruv/repos/arrow/cpp/src/arrow/testing/gtest_util.cc:223: Failure
   25: Failed
   25: Got:
   25:   [
   25:     [
   25:       99,
   25:       3
   25:     ],
   25:     [
   25:       99,
   25:       3
   25:     ]
   25:   ]
   25: Expected:
   25:   [
   25:     [
   25:       99
   25:     ],
   25:     [
   25:       3,
   25:       101,
   25:       22
   25:     ]
   25:   ]
   25: map type found!
   25: Value type: int32
   25: Key being checked: foo
   25: Item at key: 99
   25: Value being appended: 99
   25: /home/dhruv/repos/arrow/cpp/src/arrow/compute/kernels/test_util.cc:119: Failure
   25: Expected equality of these values:
   25:   input.array()->length
   25:     Which is: 4
   25:   expected->length()
   25:     Which is: 1
   25: [  FAILED  ] TestScalarNested.MapArrayLookup (9 ms)
   25: [----------] 1 test from TestScalarNested (9 ms total)
   25:
   25: [----------] Global test environment tear-down
   25: [==========] 1 test from 1 test suite ran. (9 ms total)
   25: [  PASSED  ] 0 tests.
   25: [  FAILED  ] 1 test, listed below:
   25: [  FAILED  ] TestScalarNested.MapArrayLookup
   25:
   25:  1 FAILED TEST
   25: ~/repos/arrow/cpp/out/build/ninja-debug/src/arrow/compute/kernels
   1/1 Test #25: arrow-compute-scalar-test ........***Failed    0.08 sec
   
   0% tests passed, 1 tests failed out of 1
   
   Label Time Summary:
   arrow_compute    =   0.08 sec*proc (1 test)
   unittest         =   0.08 sec*proc (1 test)
   
   Total Test time (real) =   0.08 sec
   
   The following tests FAILED:
            25 - arrow-compute-scalar-test (Failed)
   Errors while running CTest
   Output from these tests are in: /home/dhruv/repos/arrow/cpp/out/build/ninja-debug/Testing/Temporary/LastTest.log
   Use "--rerun-failed --output-on-failure" to re-run the failed cases verbosely.
   ```
   
   </details>
   
   Also, what would a kernel that takes `Type::MAP` of shape `ValueDescr::SCALAR` look like?
   




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       Yes, I was just suggesting that calculating the offset could be factored out.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       For LAST, you could keep track of the index outside the "loop" and simply overwrite it each time (though that is perhaps not as efficient), returning the final index. We could later define a visitor that goes in reverse. 

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       I would say, don't use SetScalarOutput and stick to AppendArraySlice. Allocating a Scalar for each item will be expensive especially for things like strings where you have to copy the data to a Scalar only to copy it again into an Array.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       In other words - you don't need SetScalarOutput, I realize it shares more code between the two cases, but now it's at the cost of performance. FindOneMapValueIndex is the important part to refactor since that will change when this kernel is templated, but using Scalars in the Array case is strictly overhead.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,169 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Result<std::shared_ptr<Scalar>> GetScalarOutput(KernelContext* ctx,
+                                                         const MapScalar map_scalar) {

Review comment:
       And yes, templating is for the type of the key.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       Yes, I was just suggesting that calculating the offset could be factored out.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -429,6 +429,167 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   {"*args"},
                                   "MakeStructOptions"};
 
+struct MapArrayLookupFunctor {
+  static Result<int64_t> FindOneMapValueIndex(const Array& keys, const Scalar& query_key,
+                                              const int64_t start, const int64_t end,
+                                              const bool from_back = false) {
+    if (!from_back) {
+      for (int64_t idx = start; idx < end; ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    } else {
+      for (int64_t idx = end - 1; idx >= start; --idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) return idx;
+      }
+    }
+    return -1;
+  }
+
+  static Status SetScalarOutput(const Array& keys, const Array& items, KernelContext* ctx,
+                                const MapArrayLookupOptions& options,
+                                std::shared_ptr<Scalar>& out) {
+    const Scalar& query_key = *options.query_key;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+
+      bool found_at_least_one_key = false;
+      for (int64_t idx = 0; idx < keys.length(); ++idx) {
+        ARROW_ASSIGN_OR_RAISE(std::shared_ptr<Scalar> key, keys.GetScalar(idx));
+
+        if (key->Equals(query_key)) {
+          found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), idx, 1));
+        }
+      }
+      if (!found_at_least_one_key) {
+        out = MakeNullScalar(list(items.type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out, MakeScalar(list(items.type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (options.occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(
+          int64_t key_match_idx,
+          FindOneMapValueIndex(keys, query_key, 0, keys.length(), from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out, items.GetScalar(key_match_idx));
+      } else {
+        out = MakeNullScalar(items.type());
+      }
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+    } else {
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+    }
+
+    for (int64_t map_array_idx = 0; map_array_idx < map_array.length(); ++map_array_idx) {
+      if (!map_array.IsValid(map_array_idx)) {
+        RETURN_NOT_OK(builder->AppendNull());
+        continue;
+      } else {
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        std::shared_ptr<Scalar> output;
+        RETURN_NOT_OK(SetScalarOutput(*keys, *items, ctx, options, output));

Review comment:
       Sure, but that case can just have offset=0. And I'm suggesting this since this is the code that will change the most when the kernel gets templated, so it will save some effort changing it then.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -430,81 +430,56 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                   "MakeStructOptions"};
 template <typename KeyType>
 struct MapArrayLookupFunctor {
-  static Result<int64_t> FindOneMapValueIndex(const Array& keys,
-                                              const Scalar& query_key_scalar,
-                                              const int64_t start, const int64_t end,
-                                              const bool from_back = false) {
-    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
-    int64_t index = 0;
-    int64_t match_idx = -1;
-    ARROW_UNUSED(VisitArrayValuesInline<KeyType>(
-        *keys.data(),
-        [&](decltype(query_key) key) -> Status {
-          if (index < start) {
-            ++index;
-            return Status::OK();
-          } else if (index < end) {
-            if (key == query_key) {
-              if (!from_back) {
-                match_idx = index;
-                return Status::Cancelled("Found first matching key");
-              } else {
-                match_idx = index;
-              }
-            }
-            ++index;
-            return Status::OK();
-          } else {
-            return Status::Cancelled("End reached");
-          }
-        },
-        [&]() -> Status {
-          if (index < end) {
-            ++index;
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool& from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (from_back) {
             return Status::OK();
           } else {
-            return Status::Cancelled("End reached");
+            return Status::Cancelled("Found key match for FIRST");
           }
         }));
 
-    return match_idx;
+    return match_index;
   }
 
-  static Result<std::unique_ptr<ArrayBuilder>> GetBuiltArray(
-      const Array& keys, const Array& items, const Scalar& query_key_scalar,
-      bool& found_at_least_one_key, const int64_t& start, const int64_t& end,
-      KernelContext* ctx) {
-    std::unique_ptr<ArrayBuilder> builder;
-    RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items.type(), &builder));
+  static Status BuildListOfItemsArray(const Array& keys, const Array& items,
+                                      const Scalar& query_key_scalar,
+                                      bool& found_at_least_one_key,

Review comment:
       Note: Arrow convention is to use bool* over bool&.




-- 
This is an automated message from the 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] lidavidm commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1375,20 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Finds either the FIRST, LAST, or ALL items with a key that matches the given
+/// query key in a map array.
+///
+/// Returns an array of items for FIRST and LAST, and an array of list of items for ALL.
+///
+/// \param[in] map_array to look in
+/// \param[in] options to pass a query key and choose which matching keys to return
+/// (FIRST, LAST or ALL)
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum

Review comment:
       ```suggestion
   /// \return the resulting datum
   /// \since 8.0.0
   ```

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,276 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, list_builder->Finish());
+      out->value = result->data();
+    } else { /* occurrence == FIRST || LAST */
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+      RETURN_NOT_OK(builder->Reserve(batch.length));
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+      RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                    builder.get()));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key) {
+    return Status::TypeError("map_array_lookup: query_key can't be empty.");
+  } else if (!options.query_key->is_valid) {
+    return Status::TypeError("map_array_lookup: query_key can't be null.");

Review comment:
       nit: these two probably make more sense as `Invalid` instead of `TypeError`

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,276 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));

Review comment:
       Yeah, we can inline it then.

##########
File path: docs/source/cpp/compute.rst
##########
@@ -1639,17 +1639,19 @@ in the respective option classes.
 Structural transforms
 ~~~~~~~~~~~~~~~~~~~~~
 
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| Function name       | Arity      | Input types                         | Output type      | Options class                | Notes  |
-+=====================+============+=====================================+==================+==============================+========+
-| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                              | \(1)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_flatten        | Unary      | List-like                           | List value type  |                              | \(2)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| list_parent_indices | Unary      | List-like                           | Int64            |                              | \(3)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
-| struct_field        | Unary      | Struct or Union                     | Computed         | :struct:`StructFieldOptions` | \(4)   |
-+---------------------+------------+-------------------------------------+------------------+------------------------------+--------+
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| Function name       | Arity      | Input types                         | Output type      | Options class                   | Notes  |
++=====================+============+=====================================+==================+=================================+========+
+| list_element        | Binary     | List-like (Arg 0), Integral (Arg 1) | List value type  |                                 | \(1)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_flatten        | Unary      | List-like                           | List value type  |                                 | \(2)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| list_parent_indices | Unary      | List-like                           | Int64            |                                 | \(3)   |
++---------------------+------------+-------------------------------------+------------------+---------------------------------+--------+
+| map_array_lookup    | Unary      | Map                                 | Computed         | :struct:`MapArrayLookupOptions` | \(4)   |

Review comment:
       I'm really sorry to bikeshed here, but now that I look at it…would "map_lookup" be a better kernel name? I think we only use "array" in a kernel name when the kernel only works on arrays, but this works on arrays and scalars.

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +429,276 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool* from_back) {
+    int64_t match_index = -1;
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          match_index = index;
+          if (*from_back) {
+            return Status::OK();
+          } else {
+            return Status::Cancelled("Found key match for FIRST");
+          }
+        }));
+
+    return match_index;
+  }
+
+  static Status BuildItemsArray(const Array& keys, const Array& items,
+                                const Scalar& query_key_scalar,
+                                bool* found_at_least_one_key, ArrayBuilder* builder) {
+    RETURN_NOT_OK(
+        FindMatchingIndices(keys, query_key_scalar, [&](int64_t index) -> Status {
+          *found_at_least_one_key = true;
+          RETURN_NOT_OK(builder->AppendArraySlice(*items.data(), index, 1));
+          return Status::OK();
+        }));
+    return Status::OK();
+  }
+
+  template <typename FoundItem>
+  static Status FindMatchingIndices(const Array& keys, const Scalar& query_key_scalar,
+                                    FoundItem callback) {
+    const auto query_key = UnboxScalar<KeyType>::Unbox(query_key_scalar);
+    int64_t index = 0;
+    Status status = VisitArrayValuesInline<KeyType>(
+        *keys.data(),
+        [&](decltype(query_key) key) -> Status {
+          if (key == query_key) {
+            return callback(index++);
+          }
+          ++index;
+          return Status::OK();
+        },
+        [&]() -> Status {
+          ++index;
+          return Status::OK();
+        });
+    if (!status.ok() && !status.IsCancelled()) {
+      return status;
+    }
+    return Status::OK();
+  }
+
+  static Status ExecMapArray(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+    const MapArray map_array(batch[0].array());
+
+    std::unique_ptr<ArrayBuilder> builder;
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(),
+                                list(map_array.map_type()->item_type()), &builder));
+      auto list_builder = checked_cast<ListBuilder*>(builder.get());
+      auto value_builder = list_builder->value_builder();
+
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool found_at_least_one_key = false;
+        RETURN_NOT_OK(
+            FindMatchingIndices(*keys, *query_key, [&](int64_t index) -> Status {
+              if (!found_at_least_one_key) RETURN_NOT_OK(list_builder->Append(true));
+              found_at_least_one_key = true;
+              RETURN_NOT_OK(value_builder->AppendArraySlice(*items->data(), index, 1));
+              return Status::OK();
+            }));
+        if (!found_at_least_one_key) {
+          RETURN_NOT_OK(list_builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, list_builder->Finish());
+      out->value = result->data();
+    } else { /* occurrence == FIRST || LAST */
+      RETURN_NOT_OK(
+          MakeBuilder(ctx->memory_pool(), map_array.map_type()->item_type(), &builder));
+      RETURN_NOT_OK(builder->Reserve(batch.length));
+      for (int64_t map_array_idx = 0; map_array_idx < map_array.length();
+           ++map_array_idx) {
+        if (!map_array.IsValid(map_array_idx)) {
+          RETURN_NOT_OK(builder->AppendNull());
+          continue;
+        }
+
+        auto map = map_array.value_slice(map_array_idx);
+        auto keys = checked_cast<const StructArray&>(*map).field(0);
+        auto items = checked_cast<const StructArray&>(*map).field(1);
+        bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+        ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                              GetOneMatchingIndex(*keys, *query_key, &from_back));
+
+        if (key_match_idx != -1) {
+          RETURN_NOT_OK(builder->AppendArraySlice(*items->data(), key_match_idx, 1));
+        } else {
+          RETURN_NOT_OK(builder->AppendNull());
+        }
+      }
+      ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+      out->value = result->data();
+    }
+
+    return Status::OK();
+  }
+
+  static Status ExecMapScalar(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+    const auto& query_key = options.query_key;
+    const auto& occurrence = options.occurrence;
+
+    std::shared_ptr<DataType> item_type =
+        checked_cast<const MapType&>(*batch[0].type()).item_type();
+    const auto& map_scalar = batch[0].scalar_as<MapScalar>();
+
+    if (ARROW_PREDICT_FALSE(!map_scalar.is_valid)) {
+      if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+        out->value = MakeNullScalar(list(item_type));
+      } else {
+        out->value = MakeNullScalar(item_type);
+      }
+      return Status::OK();
+    }
+
+    const auto& struct_array = checked_cast<const StructArray&>(*map_scalar.value);
+    const std::shared_ptr<Array> keys = struct_array.field(0);
+    const std::shared_ptr<Array> items = struct_array.field(1);
+
+    if (occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+      bool found_at_least_one_key = false;
+      std::unique_ptr<ArrayBuilder> builder;
+      RETURN_NOT_OK(MakeBuilder(ctx->memory_pool(), items->type(), &builder));
+      RETURN_NOT_OK(BuildItemsArray(*keys, *items, *query_key, &found_at_least_one_key,
+                                    builder.get()));
+
+      if (!found_at_least_one_key) {
+        out->value = MakeNullScalar(list(items->type()));
+      } else {
+        ARROW_ASSIGN_OR_RAISE(auto result, builder->Finish());
+        ARROW_ASSIGN_OR_RAISE(out->value, MakeScalar(list(items->type()), result));
+      }
+    } else { /* occurrence == FIRST || LAST */
+      bool from_back = (occurrence == MapArrayLookupOptions::LAST);
+
+      ARROW_ASSIGN_OR_RAISE(int64_t key_match_idx,
+                            GetOneMatchingIndex(*keys, *query_key, &from_back));
+      if (key_match_idx != -1) {
+        ARROW_ASSIGN_OR_RAISE(out->value, items->GetScalar(key_match_idx));
+      } else {
+        out->value = MakeNullScalar(items->type());
+      }
+    }
+    return Status::OK();
+  }
+};
+
+Result<ValueDescr> ResolveMapArrayLookupType(KernelContext* ctx,
+                                             const std::vector<ValueDescr>& descrs) {
+  const auto& options = OptionsWrapper<MapArrayLookupOptions>::Get(ctx);
+  std::shared_ptr<DataType> type = descrs.front().type;
+  std::shared_ptr<DataType> item_type = checked_cast<const MapType&>(*type).item_type();
+  std::shared_ptr<DataType> key_type = checked_cast<const MapType&>(*type).key_type();
+
+  if (!options.query_key) {
+    return Status::TypeError("map_array_lookup: query_key can't be empty.");
+  } else if (!options.query_key->is_valid) {
+    return Status::TypeError("map_array_lookup: query_key can't be null.");
+  } else if (!options.query_key->type || !options.query_key->type->Equals(key_type)) {
+    return Status::TypeError(
+        "map_array_lookup: query_key type and MapArray key_type don't match. Expected "
+        "type: ",
+        *key_type, ", but got type: ", *options.query_key->type);
+  }
+
+  if (options.occurrence == MapArrayLookupOptions::Occurrence::ALL) {
+    return ValueDescr(list(item_type), descrs.front().shape);
+  } else { /* occurrence == FIRST || LAST */
+    return ValueDescr(item_type, descrs.front().shape);
+  }
+}
+
+struct ResolveMapArrayLookup {
+  KernelContext* ctx;
+  const ExecBatch& batch;
+  Datum* out;
+
+  template <typename KeyType>
+  Status Execute() {
+    if (batch[0].kind() == Datum::SCALAR) {
+      return MapArrayLookupFunctor<KeyType>::ExecMapScalar(ctx, batch, out);
+    }
+    return MapArrayLookupFunctor<KeyType>::ExecMapArray(ctx, batch, out);
+  }
+
+  template <typename KeyType>
+  enable_if_physical_integer<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_decimal<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_base_binary<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_boolean<KeyType, Status> Visit(const KeyType& type) {
+    return Execute<KeyType>();
+  }
+
+  template <typename KeyType>
+  enable_if_same<KeyType, FixedSizeBinaryType, Status> Visit(const KeyType& key) {
+    return Execute<KeyType>();
+  }
+
+  Status Visit(const MonthDayNanoIntervalType& key) {
+    return Execute<MonthDayNanoIntervalType>();
+  }
+
+  Status Visit(const DataType& type) {
+    return Status::TypeError("Got unsupported type: ", type.ToString());
+  }
+
+  static Status Exec(KernelContext* ctx, const ExecBatch& batch, Datum* out) {
+    ResolveMapArrayLookup visitor{ctx, batch, out};
+    return VisitTypeInline(*checked_cast<const MapType&>(*batch[0].type()).key_type(),
+                           &visitor);
+  }
+};
+
+void AddMapArrayLookupKernels(ScalarFunction* func) {
+  for (const auto shape : {ValueDescr::ARRAY, ValueDescr::SCALAR}) {
+    ScalarKernel kernel(
+        {InputType(Type::MAP, shape)}, OutputType(ResolveMapArrayLookupType),
+        ResolveMapArrayLookup::Exec, OptionsWrapper<MapArrayLookupOptions>::Init);

Review comment:
       Since the Exec is the same regardless of the shape, we can remove the loop and just register one kernel for `InputType(Type::MAP)`

##########
File path: python/pyarrow/_compute.pyx
##########
@@ -1333,6 +1333,43 @@ class IndexOptions(_IndexOptions):
         self._set_options(value)
 
 
+cdef class _MapArrayLookupOptions(FunctionOptions):
+    _occurrence_map = {
+        "ALL": CMapArrayLookupOccurrence_ALL,
+        "FIRST": CMapArrayLookupOccurrence_FIRST,
+        "LAST": CMapArrayLookupOccurrence_LAST,
+    }
+
+    def _set_options(self, scalar, occurrence):
+        try:
+            self.wrapped.reset(
+                new CMapArrayLookupOptions(
+                    pyarrow_unwrap_scalar(scalar),
+                    self._occurrence_map[occurrence]
+                )
+            )
+        except KeyError:
+            _raise_invalid_function_option(occurrence,
+                                           "Should either be FIRST, LAST or ALL")
+
+
+class MapArrayLookupOptions(_MapArrayLookupOptions):
+    """
+    Options for the `map_array_lookup` function.
+
+    Parameters
+    ----------
+    query_key : Scalar
+        The key to search for.
+    occurrence : str
+        The occurrence(s) to return from the MapArray
+        Accepted values are "FIRST", "LAST", "ALL".

Review comment:
       nit, we usually take lowercase options strings, e.g. see the round modes: https://github.com/apache/arrow/blob/10d108d6f0fae2a5265116ff23fe7a0623c2dc2f/python/pyarrow/_compute.pyx#L793-L814




-- 
This is an automated message from the 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] dhruv9vats commented on a change in pull request #12162: ARROW-15089: [C++][Compute] Implement kernel to lookup a MapArray item for a given key

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



##########
File path: cpp/src/arrow/compute/api_scalar.h
##########
@@ -1350,5 +1375,20 @@ ARROW_EXPORT Result<Datum> AssumeTimezone(const Datum& values,
                                           AssumeTimezoneOptions options,
                                           ExecContext* ctx = NULLPTR);
 
+/// \brief Finds either the FIRST, LAST, or ALL items with a key that matches the given
+/// query key in a map array.
+///
+/// Returns an array of items for FIRST and LAST, and an array of list of items for ALL.
+///
+/// \param[in] map_array to look in
+/// \param[in] options to pass a query key and choose which matching keys to return
+/// (FIRST, LAST or ALL)
+/// \param[in] ctx the function execution context, optional
+///
+/// \return the resulting datum
+/// \note API not yet finalized
+ARROW_EXPORT Result<Datum> MapArrayLookup(const Datum& map_array,
+                                          MapArrayLookupOptions options,
+                                          ExecContext* ctx = NULLPTR);

Review comment:
       In a sync with Weston earlier this week, he mentioned adding this to help with IDE autocompletion. Is this correct?

##########
File path: cpp/src/arrow/compute/kernels/scalar_nested.cc
##########
@@ -428,6 +428,273 @@ const FunctionDoc make_struct_doc{"Wrap Arrays into a StructArray",
                                    "specified through MakeStructOptions."),
                                   {"*args"},
                                   "MakeStructOptions"};
+template <typename KeyType>
+struct MapArrayLookupFunctor {
+  static Result<int64_t> GetOneMatchingIndex(const Array& keys,
+                                             const Scalar& query_key_scalar,
+                                             const bool& from_back) {

Review comment:
       My (probably naive) line of thought:
   - Still differentiating the helper functions on the bases of options (`ALL` and `FIRST`/`LAST`) as opposed to input shapes as hinted by you as we need to iterate till the end for the `LAST` option and putting that logic into a helper function will fill it with conditional statements, making it look similar to what `ExecMapArray` looks right now.
   - Also still returning an `int` instead of appending to an array builder as for the scalar case, as just a scalar via `GetScalar` gets returned which, too, will introduce conditional statements.

##########
File path: cpp/src/arrow/compute/api_scalar.cc
##########
@@ -785,5 +819,12 @@ Result<Datum> Week(const Datum& arg, WeekOptions options, ExecContext* ctx) {
   return CallFunction("week", {arg}, &options, ctx);
 }
 
+// ----------------------------------------------------------------------
+
+Result<Datum> MapArrayLookup(const Datum& arg, MapArrayLookupOptions options,
+                             ExecContext* ctx) {
+  return CallFunction("map_array_lookup", {arg}, &options, ctx);
+}
+

Review comment:
       Where should this go?




-- 
This is an automated message from the 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